You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by su...@apache.org on 2016/04/07 03:07:13 UTC

[1/4] drill git commit: DRILL-3743: Fail active result listeners if server connection is closed

Repository: drill
Updated Branches:
  refs/heads/master e7e9b73c1 -> 9514cbe75


DRILL-3743: Fail active result listeners if server connection is closed

+ Remove dead code
+ Improve error and logging messages

closes #460


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/55d54eda
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/55d54eda
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/55d54eda

Branch: refs/heads/master
Commit: 55d54eda392961d858c171b737c6ce76cf9a9fb1
Parents: e7e9b73
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Tue Apr 5 16:04:44 2016 -0700
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Wed Apr 6 11:37:36 2016 -0700

----------------------------------------------------------------------
 .../drill/exec/rpc/user/QueryResultHandler.java | 126 +++++++++++--------
 .../apache/drill/exec/rpc/user/UserClient.java  |   7 +-
 2 files changed, 78 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/55d54eda/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
index ca73ac8..00a324b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.rpc.user;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
-import io.netty.channel.ChannelFuture;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.GenericFutureListener;
 
@@ -35,9 +34,10 @@ import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
+import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
 import org.apache.drill.exec.rpc.ConnectionThrottle;
-import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 
@@ -69,9 +69,13 @@ public class QueryResultHandler {
   private final ConcurrentMap<QueryId, UserResultsListener> queryIdToResultsListenersMap =
       Maps.newConcurrentMap();
 
-  public RpcOutcomeListener<QueryId> getWrappedListener(RemoteConnection connection,
-      UserResultsListener resultsListener) {
-    return new SubmissionListener(connection, resultsListener);
+  public RpcOutcomeListener<QueryId> getWrappedListener(UserResultsListener resultsListener) {
+    return new SubmissionListener(resultsListener);
+  }
+
+  public RpcConnectionHandler<ServerConnection> getWrappedConnectionHandler(
+      final RpcConnectionHandler<ServerConnection> handler) {
+    return new ChannelClosedHandler(handler);
   }
 
   /**
@@ -84,7 +88,10 @@ public class QueryResultHandler {
     final QueryId queryId = queryResult.getQueryId();
     final QueryState queryState = queryResult.getQueryState();
 
-    logger.debug( "resultArrived: queryState: {}, queryId = {}", queryState, queryId );
+    if (logger.isDebugEnabled()) {
+      logger.debug("resultArrived: queryState: {}, queryId = {}", queryState,
+          QueryIdHelper.getQueryId(queryId));
+    }
 
     assert queryResult.hasQueryState() : "received query result without QueryState";
 
@@ -92,9 +99,6 @@ public class QueryResultHandler {
     // CANCELED queries are handled the same way as COMPLETED
     final boolean isTerminalResult;
     switch ( queryState ) {
-      case STARTING:
-        isTerminalResult = false;
-        break;
       case FAILED:
       case CANCELED:
       case COMPLETED:
@@ -154,7 +158,9 @@ public class QueryResultHandler {
 
     final QueryId queryId = queryData.getQueryId();
 
-    logger.debug( "batchArrived: queryId = {}", queryId );
+    if (logger.isDebugEnabled()) {
+      logger.debug("batchArrived: queryId = {}", QueryIdHelper.getQueryId(queryId));
+    }
     logger.trace( "batchArrived: batch = {}", batch );
 
     final UserResultsListener resultsListener = newUserResultsListener(queryId);
@@ -189,20 +195,10 @@ public class QueryResultHandler {
       if ( null == resultsListener ) {
         resultsListener = bl;
       }
-      // TODO:  Is there a more direct way to detect a Query ID in whatever state this string comparison detects?
-      if ( queryId.toString().isEmpty() ) {
-        failAll();
-      }
     }
     return resultsListener;
   }
 
-  private void failAll() {
-    for (UserResultsListener l : queryIdToResultsListenersMap.values()) {
-      l.submissionFailed(UserException.systemError(new RpcException("Received result without QueryId")).build(logger));
-    }
-  }
-
   private static class BufferingResultsListener implements UserResultsListener {
 
     private ConcurrentLinkedQueue<QueryDataBatch> results = Queues.newConcurrentLinkedQueue();
@@ -272,55 +268,41 @@ public class QueryResultHandler {
     @Override
     public void queryIdArrived(QueryId queryId) {
     }
-
   }
 
-
   private class SubmissionListener extends BaseRpcOutcomeListener<QueryId> {
+
     private final UserResultsListener resultsListener;
-    private final RemoteConnection connection;
-    private final ChannelFuture closeFuture;
-    private final ChannelClosedListener closeListener;
     private final AtomicBoolean isTerminal = new AtomicBoolean(false);
 
-    public SubmissionListener(RemoteConnection connection, UserResultsListener resultsListener) {
-      super();
+    public SubmissionListener(UserResultsListener resultsListener) {
       this.resultsListener = resultsListener;
-      this.connection = connection;
-      this.closeFuture = connection.getChannel().closeFuture();
-      this.closeListener = new ChannelClosedListener();
-      closeFuture.addListener(closeListener);
-    }
-
-    private class ChannelClosedListener implements GenericFutureListener<Future<Void>> {
-
-      @Override
-      public void operationComplete(Future<Void> future) throws Exception {
-        resultsListener.submissionFailed(UserException.connectionError()
-            .message("Connection %s closed unexpectedly.", connection.getName())
-            .build(logger));
-      }
-
     }
 
     @Override
     public void failed(RpcException ex) {
       if (!isTerminal.compareAndSet(false, true)) {
+        logger.warn("Received multiple responses to run query request.");
         return;
       }
 
-      closeFuture.removeListener(closeListener);
-      resultsListener.submissionFailed(UserException.systemError(ex).build(logger));
-
+      // Although query submission failed, results might have arrived for this query.
+      // However, the results could not be transferred to this resultListener because
+      // there is no query id mapped to this resultListener. Look out for the warning
+      // message from ChannelClosedHandler in the client logs.
+      // TODO(DRILL-4586)
+      resultsListener.submissionFailed(UserException.systemError(ex)
+          .addContext("Query submission to Drillbit failed.")
+          .build(logger));
     }
 
     @Override
     public void success(QueryId queryId, ByteBuf buf) {
       if (!isTerminal.compareAndSet(false, true)) {
+        logger.warn("Received multiple responses to run query request.");
         return;
       }
 
-      closeFuture.removeListener(closeListener);
       resultsListener.queryIdArrived(queryId);
       if (logger.isDebugEnabled()) {
         logger.debug("Received QueryId {} successfully. Adding results listener {}.",
@@ -354,17 +336,57 @@ public class QueryResultHandler {
 
     @Override
     public void interrupted(final InterruptedException ex) {
-      logger.warn("Interrupted while waiting for query results from Drillbit", ex);
-
       if (!isTerminal.compareAndSet(false, true)) {
+        logger.warn("Received multiple responses to run query request.");
         return;
       }
 
-      closeFuture.removeListener(closeListener);
-
-      // Throw an interrupted UserException?
-      resultsListener.submissionFailed(UserException.systemError(ex).build(logger));
+      // TODO(DRILL-4586)
+      resultsListener.submissionFailed(UserException.systemError(ex)
+          .addContext("The client had been asked to wait as the Drillbit is potentially being over-utilized." +
+              " But the client was interrupted while waiting.")
+          .build(logger));
     }
   }
 
+  /**
+   * When a {@link ServerConnection connection} to a server is successfully created, this handler adds a
+   * listener to that connection that listens to connection closure. If the connection is closed, all active
+   * {@link UserResultsListener result listeners} are failed.
+   */
+  private class ChannelClosedHandler implements RpcConnectionHandler<ServerConnection> {
+
+    private final RpcConnectionHandler<ServerConnection> parentHandler;
+
+    public ChannelClosedHandler(final RpcConnectionHandler<ServerConnection> parentHandler) {
+      this.parentHandler = parentHandler;
+    }
+
+    @Override
+    public void connectionSucceeded(final ServerConnection connection) {
+      connection.getChannel().closeFuture().addListener(
+          new GenericFutureListener<Future<? super Void>>() {
+            @Override
+            public void operationComplete(Future<? super Void> future)
+                throws Exception {
+              for (final UserResultsListener listener : queryIdToResultsListenersMap.values()) {
+                listener.submissionFailed(UserException.connectionError()
+                    .message("Connection %s closed unexpectedly. Drillbit down?",
+                        connection.getName())
+                    .build(logger));
+                if (listener instanceof BufferingResultsListener) {
+                  // the appropriate listener will be failed by SubmissionListener#failed
+                  logger.warn("Buffering listener failed before results were transferred to the actual listener.");
+                }
+              }
+            }
+          });
+      parentHandler.connectionSucceeded(connection);
+    }
+
+    @Override
+    public void connectionFailed(FailureType type, Throwable t) {
+      parentHandler.connectionFailed(type, t);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/55d54eda/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index 824e6eb..5ff6a6d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -67,11 +67,11 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
   }
 
   public void submitQuery(UserResultsListener resultsListener, RunQuery query) {
-    send(queryResultHandler.getWrappedListener(connection, resultsListener), RpcType.RUN_QUERY, query, QueryId.class);
+    send(queryResultHandler.getWrappedListener(resultsListener), RpcType.RUN_QUERY, query, QueryId.class);
   }
 
   public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
-      UserProperties props, UserBitShared.UserCredentials credentials) {
+                      UserProperties props, UserBitShared.UserCredentials credentials) {
     UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder()
         .setRpcVersion(UserRpcConfig.RPC_VERSION)
         .setSupportListening(true)
@@ -83,7 +83,8 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
       hsBuilder.setProperties(props);
     }
 
-    this.connectAsClient(handler, hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+    this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
+        hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
   }
 
   @Override


[4/4] drill git commit: DRILL-4544: Improve error messages for REFRESH TABLE METADATA command 1. Added error message when storage plugin or workspace does not exist 2. Updated error message when refresh metadata is not supported 3. Unit tests

Posted by su...@apache.org.
DRILL-4544: Improve error messages for REFRESH TABLE METADATA command
1. Added error message when storage plugin or workspace does not exist
2. Updated error message when refresh metadata is not supported
3. Unit tests

closes #448


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/a7595525
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/a7595525
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/a7595525

Branch: refs/heads/master
Commit: a7595525419935e40cef75251b43b946e0cf79ec
Parents: 68269af
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Mon Mar 28 10:55:56 2016 +0000
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Wed Apr 6 16:57:40 2016 -0700

----------------------------------------------------------------------
 .../sql/handlers/RefreshMetadataHandler.java    |  8 +++++-
 .../store/parquet/TestParquetMetadataCache.java | 28 ++++++++++++++++++++
 2 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a7595525/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
index 059f7d0..7be46f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
@@ -30,6 +30,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.parser.SqlRefreshMetadata;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
@@ -52,7 +53,7 @@ public class RefreshMetadataHandler extends DefaultSqlHandler {
   }
 
   private PhysicalPlan notSupported(String tbl){
-    return direct(false, "Table %s does not support metadata refresh.  Support is currently limited to single-directory-based Parquet tables.", tbl);
+    return direct(false, "Table %s does not support metadata refresh. Support is currently limited to directory-based Parquet tables.", tbl);
   }
 
   @Override
@@ -64,6 +65,11 @@ public class RefreshMetadataHandler extends DefaultSqlHandler {
       final SchemaPlus schema = findSchema(config.getConverter().getDefaultSchema(),
           refreshTable.getSchemaPath());
 
+      if (schema == null) {
+        return direct(false, "Storage plugin or workspace does not exist [%s]",
+            SchemaUtilites.SCHEMA_PATH_JOINER.join(refreshTable.getSchemaPath()));
+      }
+
       final String tableName = refreshTable.getName();
 
       if (tableName.contains("*") || tableName.contains("?")) {

http://git-wip-us.apache.org/repos/asf/drill/blob/a7595525/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
index 4330c96..afcea87 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
@@ -177,6 +177,34 @@ public class TestParquetMetadataCache extends PlanTestBase {
       .go();
   }
 
+  @Test
+  public void testAbsentPluginOrWorkspaceError() throws Exception {
+    testBuilder()
+        .sqlQuery("refresh table metadata dfs_test.incorrect.table_name")
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, "Storage plugin or workspace does not exist [dfs_test.incorrect]")
+        .go();
+
+    testBuilder()
+        .sqlQuery("refresh table metadata incorrect.table_name")
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, "Storage plugin or workspace does not exist [incorrect]")
+        .go();
+  }
+
+  @Test
+  public void testNoSupportedError() throws Exception {
+    testBuilder()
+        .sqlQuery("refresh table metadata cp.`tpch/nation.parquet`")
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, "Table tpch/nation.parquet does not support metadata refresh. " +
+            "Support is currently limited to directory-based Parquet tables.")
+        .go();
+  }
+
   private void checkForMetadataFile(String table) throws Exception {
     String tmpDir = getDfsTestTmpSchemaLocation();
     String metaFile = Joiner.on("/").join(tmpDir, table, Metadata.METADATA_FILENAME);


[3/4] drill git commit: DRILL-4523: Disallow using loopback address in distributed mode

Posted by su...@apache.org.
DRILL-4523: Disallow using loopback address in distributed mode

closes #445


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/9514cbe7
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/9514cbe7
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/9514cbe7

Branch: refs/heads/master
Commit: 9514cbe7574a5bfce40bf8fe982faf2aef48bc0e
Parents: a759552
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Mar 24 16:46:06 2016 +0000
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Wed Apr 6 16:57:40 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/drill/exec/server/Drillbit.java    |  9 ++++++---
 .../org/apache/drill/exec/service/ServiceEngine.java   | 13 ++++++++++++-
 2 files changed, 18 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/9514cbe7/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 441fa91..d981342 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -86,18 +86,21 @@ public class Drillbit implements AutoCloseable {
     final boolean allowPortHunting = serviceSet != null;
     context = new BootStrapContext(config, classpathScan);
     manager = new WorkManager(context);
-    engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context,
-        manager.getWorkBus(), manager.getBee(), allowPortHunting);
 
     webServer = new WebServer(config, context.getMetrics(), manager);
-
+    boolean isDistributedMode = false;
     if (serviceSet != null) {
       coord = serviceSet.getCoordinator();
       storeProvider = new CachingPersistentStoreProvider(new LocalPersistentStoreProvider(config));
     } else {
       coord = new ZKClusterCoordinator(config);
       storeProvider = new PersistentStoreRegistry(this.coord, config).newPStoreProvider();
+      isDistributedMode = true;
     }
+
+    engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context,
+        manager.getWorkBus(), manager.getBee(), allowPortHunting, isDistributedMode);
+
     logger.info("Construction completed ({} ms).", w.elapsed(TimeUnit.MILLISECONDS));
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9514cbe7/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index 169e26e..17edbc2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
@@ -59,13 +60,14 @@ public class ServiceEngine implements AutoCloseable {
   private final DrillConfig config;
   boolean useIP = false;
   private final boolean allowPortHunting;
+  private final boolean isDistributedMode;
   private final BufferAllocator userAllocator;
   private final BufferAllocator controlAllocator;
   private final BufferAllocator dataAllocator;
 
 
   public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context,
-      WorkEventBus workBus, WorkerBee bee, boolean allowPortHunting) throws DrillbitStartupException {
+      WorkEventBus workBus, WorkerBee bee, boolean allowPortHunting, boolean isDistributedMode) throws DrillbitStartupException {
     userAllocator = newAllocator(context, "rpc:user", "drill.exec.rpc.user.server.memory.reservation",
         "drill.exec.rpc.user.server.memory.maximum");
     controlAllocator = newAllocator(context, "rpc:bit-control",
@@ -85,6 +87,7 @@ public class ServiceEngine implements AutoCloseable {
     this.dataPool = new DataConnectionCreator(context, dataAllocator, workBus, bee, allowPortHunting);
     this.config = context.getConfig();
     this.allowPortHunting = allowPortHunting;
+    this.isDistributedMode = isDistributedMode;
     registerMetrics(context.getMetrics());
 
   }
@@ -141,6 +144,8 @@ public class ServiceEngine implements AutoCloseable {
   public DrillbitEndpoint start() throws DrillbitStartupException, UnknownHostException{
     int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT), allowPortHunting);
     String address = useIP ?  InetAddress.getLocalHost().getHostAddress() : InetAddress.getLocalHost().getCanonicalHostName();
+    checkLoopbackAddress(address);
+
     DrillbitEndpoint partialEndpoint = DrillbitEndpoint.newBuilder()
         .setAddress(address)
         //.setAddress("localhost")
@@ -177,6 +182,12 @@ public class ServiceEngine implements AutoCloseable {
     });
   }
 
+  private void checkLoopbackAddress(String address) throws DrillbitStartupException, UnknownHostException {
+    if (isDistributedMode && InetAddress.getByName(address).isLoopbackAddress()) {
+      throw new DrillbitStartupException("Drillbit is disallowed to bind to loopback address in distributed mode.");
+    }
+  }
+
   @Override
   public void close() throws Exception {
     // this takes time so close them in parallel


[2/4] drill git commit: DRILL-4588: Enable JMX reporting

Posted by su...@apache.org.
DRILL-4588: Enable JMX reporting

closes #469


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/68269af3
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/68269af3
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/68269af3

Branch: refs/heads/master
Commit: 68269af37b4f7b7de44b32e23f63633983b63921
Parents: 55d54ed
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Wed Apr 6 16:41:52 2016 -0700
Committer: Sudheesh Katkam <sk...@maprtech.com>
Committed: Wed Apr 6 16:57:40 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/drill/exec/metrics/DrillMetrics.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/68269af3/common/src/main/java/org/apache/drill/exec/metrics/DrillMetrics.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/exec/metrics/DrillMetrics.java b/common/src/main/java/org/apache/drill/exec/metrics/DrillMetrics.java
index e99a40f..568a97b 100644
--- a/common/src/main/java/org/apache/drill/exec/metrics/DrillMetrics.java
+++ b/common/src/main/java/org/apache/drill/exec/metrics/DrillMetrics.java
@@ -48,13 +48,13 @@ public class DrillMetrics {
 
   private static class RegistryHolder {
     public static final MetricRegistry REGISTRY;
-    // private static final JmxReporter JMX_REPORTER;
+    private static final JmxReporter JMX_REPORTER;
     private static final Slf4jReporter LOG_REPORTER;
 
     static {
       REGISTRY = new MetricRegistry();
       registerSysStats();
-      // JMX_REPORTER = getJmxReporter();
+      JMX_REPORTER = getJmxReporter();
       LOG_REPORTER = getLogReporter();
     }