|
@@ -22,9 +22,14 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.util.Random;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
+import com.google.common.base.Optional;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -41,7 +46,6 @@ import org.apache.hadoop.ozone.storage.StorageContainerManager;
|
|
|
import org.apache.hadoop.ozone.web.client.OzoneClient;
|
|
|
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
-import org.apache.hadoop.util.Time;
|
|
|
|
|
|
/**
|
|
|
* MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
|
|
@@ -82,6 +86,7 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
|
|
|
extends org.apache.hadoop.hdfs.MiniDFSCluster.Builder {
|
|
|
|
|
|
private final OzoneConfiguration conf;
|
|
|
+ private Optional<String> ozoneHandlerType = Optional.absent();
|
|
|
|
|
|
/**
|
|
|
* Creates a new Builder.
|
|
@@ -100,13 +105,35 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
+ public Builder setHandlerType(String handler) {
|
|
|
+ ozoneHandlerType = Optional.of(handler);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public MiniOzoneCluster build() throws IOException {
|
|
|
+ if (!ozoneHandlerType.isPresent()) {
|
|
|
+ throw new IllegalArgumentException(
|
|
|
+ "The Ozone handler type must be specified.");
|
|
|
+ }
|
|
|
+
|
|
|
+ conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
|
|
|
+ conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY, true);
|
|
|
+ conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, ozoneHandlerType.get());
|
|
|
conf.set(OzoneConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
|
|
|
conf.set(OzoneConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
|
|
|
StorageContainerManager scm = new StorageContainerManager(conf);
|
|
|
scm.start();
|
|
|
- return new MiniOzoneCluster(this, scm);
|
|
|
+ MiniOzoneCluster cluster = new MiniOzoneCluster(this, scm);
|
|
|
+ try {
|
|
|
+ cluster.waitOzoneReady();
|
|
|
+ } catch(Exception e) {
|
|
|
+ // A workaround to propagate MiniOzoneCluster failures without
|
|
|
+ // changing the method signature (which would require cascading
|
|
|
+ // changes to hundreds of unrelated HDFS tests).
|
|
|
+ throw new IOException("Failed to start MiniOzoneCluster", e);
|
|
|
+ }
|
|
|
+ return cluster;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -176,22 +203,19 @@ public class MiniOzoneCluster extends MiniDFSCluster implements Closeable {
|
|
|
/**
|
|
|
* Waits for the Ozone cluster to be ready for processing requests.
|
|
|
*/
|
|
|
- public void waitOzoneReady() {
|
|
|
- long begin = Time.monotonicNow();
|
|
|
- while (scm.getDatanodeReport(DatanodeReportType.LIVE).length <
|
|
|
- numDataNodes) {
|
|
|
- if (Time.monotonicNow() - begin > 20000) {
|
|
|
- throw new IllegalStateException(
|
|
|
- "Timed out waiting for Ozone cluster to become ready.");
|
|
|
+ public void waitOzoneReady() throws TimeoutException, InterruptedException {
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ final DatanodeInfo[] reports =
|
|
|
+ scm.getDatanodeReport(DatanodeReportType.LIVE);
|
|
|
+ if (reports.length >= numDataNodes) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ LOG.info("Waiting for cluster to be ready. Got {} of {} DN reports.",
|
|
|
+ reports.length, numDataNodes);
|
|
|
+ return false;
|
|
|
}
|
|
|
- LOG.info("Waiting for Ozone cluster to become ready");
|
|
|
- try {
|
|
|
- Thread.sleep(100);
|
|
|
- } catch (InterruptedException e) {
|
|
|
- Thread.currentThread().interrupt();
|
|
|
- throw new IllegalStateException(
|
|
|
- "Interrupted while waiting for Ozone cluster to become ready.");
|
|
|
- }
|
|
|
- }
|
|
|
+ }, 100, 45000);
|
|
|
}
|
|
|
}
|