You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/02/09 16:59:20 UTC

[01/21] impala git commit: IMPALA-6228: Control stats extrapolation via tbl prop.

Repository: impala
Updated Branches:
  refs/heads/2.x fbe637f6e -> bf892858a


IMPALA-6228: Control stats extrapolation via tbl prop.

Introduces a new TBLPROPERTY for controlling stats
extrapolation on a per-table basis:

impala.enable.stats.extrapolation=true/false

The property key was chosen to be consistent with
the impalad startup flag --enable_stats_extrapolation
and to indicate that the property was set and is used
by Impala.

Behavior:
- If the property is not set, then the extrapolation
  behavior is determined by the impalad startup flag.
- If the property is set, it overrides the impalad
  startup flag, i.e., extrapolation can be explicitly
  enabled or disabled regardless of the startup flag.

Testing:
- added new unit tests
- code/hdfs run passed

Change-Id: Ie49597bf1b93b7572106abc620d91f199cba0cfd
Reviewed-on: http://gerrit.cloudera.org:8080/9139
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins
Reviewed-on: http://gerrit.cloudera.org:8080/9231
Reviewed-by: Philip Zeyliger <ph...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: d181610614de0132e7dc2f3e98f99ba1cd2377cf
Parents: fbe637f
Author: Alex Behm <al...@cloudera.com>
Authored: Wed Jan 24 11:58:53 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Feb 7 22:22:00 2018 +0000

----------------------------------------------------------------------
 .../impala/analysis/ComputeStatsStmt.java       |  26 ++-
 .../org/apache/impala/catalog/HdfsTable.java    |  22 ++-
 .../org/apache/impala/planner/HdfsScanNode.java |   2 +-
 .../apache/impala/service/BackendConfig.java    |   2 +-
 .../apache/impala/analysis/AnalyzeDDLTest.java  |  80 ++++++---
 .../impala/planner/StatsExtrapolationTest.java  |  91 +++++++---
 .../queries/QueryTest/stats-extrapolation.test  |   4 +-
 .../custom_cluster/test_stats_extrapolation.py  | 135 ++------------
 tests/metadata/test_stats_extrapolation.py      | 175 +++++++++++++++++++
 9 files changed, 361 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java b/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
index 6ca8dc9..54daf7f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ComputeStatsStmt.java
@@ -52,7 +52,10 @@ import com.google.common.collect.Sets;
  * Represents the following statements for statistics collection. Which statistics
  * are computed and stored depends on the statement type (incremental or not), the
  * clauses used (sampling, partition spec), as well as whether stats extrapolation
- * is enabled or not (--enable_stats_extrapolation).
+ * is enabled or not.
+ * Stats extrapolation can be configured:
+ * - at the impalad level with --enable_stats_extrapolation
+ * - at the table level HdfsTable.TBL_PROP_ENABLE_STATS_EXTRAPOLATION
  *
  * 1. COMPUTE STATS <table> [(col_list)] [TABLESAMPLE SYSTEM(<perc>) [REPEATABLE(<seed>)]]
  * - Stats extrapolation enabled:
@@ -481,8 +484,10 @@ public class ComputeStatsStmt extends StatementBase {
       }
     } else {
       // Not computing incremental stats.
-      expectAllPartitions_ = !(table_ instanceof HdfsTable) ||
-          !BackendConfig.INSTANCE.enableStatsExtrapolation();
+      expectAllPartitions_ = true;
+      if (table_ instanceof HdfsTable) {
+        expectAllPartitions_ = !((HdfsTable) table_).isStatsExtrapolationEnabled();
+      }
     }
 
     if (filterPreds.size() > MAX_INCREMENTAL_PARTITIONS) {
@@ -576,10 +581,14 @@ public class ComputeStatsStmt extends StatementBase {
     if (!(table_ instanceof HdfsTable)) {
       throw new AnalysisException("TABLESAMPLE is only supported on HDFS tables.");
     }
-    if (!BackendConfig.INSTANCE.enableStatsExtrapolation()) {
-      throw new AnalysisException(
-          "COMPUTE STATS TABLESAMPLE requires --enable_stats_extrapolation=true. " +
-          "Stats extrapolation is currently disabled.");
+    HdfsTable hdfsTable = (HdfsTable) table_;
+    if (!hdfsTable.isStatsExtrapolationEnabled()) {
+      throw new AnalysisException(String.format(
+          "COMPUTE STATS TABLESAMPLE requires stats extrapolation which is disabled.\n" +
+          "Stats extrapolation can be enabled service-wide with %s=true or by altering " +
+          "the table to have tblproperty %s=true",
+          "--enable_stats_extrapolation",
+          HdfsTable.TBL_PROP_ENABLE_STATS_EXTRAPOLATION));
     }
     sampleParams_.analyze(analyzer);
     long sampleSeed;
@@ -592,7 +601,6 @@ public class ComputeStatsStmt extends StatementBase {
     // Compute the sample of files and set 'sampleFileBytes_'.
     long minSampleBytes = analyzer.getQueryOptions().compute_stats_min_sample_size;
     long samplePerc = sampleParams_.getPercentBytes();
-    HdfsTable hdfsTable = (HdfsTable) table_;
     Map<Long, List<FileDescriptor>> sample = hdfsTable.getFilesSample(
         hdfsTable.getPartitions(), samplePerc, minSampleBytes, sampleSeed);
     long sampleFileBytes = 0;
@@ -696,7 +704,7 @@ public class ComputeStatsStmt extends StatementBase {
    */
   private boolean updateTableStatsOnly() {
     if (!(table_ instanceof HdfsTable)) return true;
-    return !isIncremental_ && BackendConfig.INSTANCE.enableStatsExtrapolation();
+    return !isIncremental_ && ((HdfsTable) table_).isStatsExtrapolationEnabled();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index adc6aef..0f782be 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -126,6 +126,12 @@ public class HdfsTable extends Table {
   // Table property key for skip.header.line.count
   public static final String TBL_PROP_SKIP_HEADER_LINE_COUNT = "skip.header.line.count";
 
+  // Table property key for overriding the Impalad-wide --enable_stats_extrapolation
+  // setting for a specific table. By default, tables do not have the property set and
+  // rely on the Impalad-wide --enable_stats_extrapolation flag.
+  public static final String TBL_PROP_ENABLE_STATS_EXTRAPOLATION =
+      "impala.enable.stats.extrapolation";
+
   // Average memory requirements (in bytes) for storing the metadata of a partition.
   private static final long PER_PARTITION_MEM_USAGE_BYTES = 2048;
 
@@ -1951,7 +1957,7 @@ public class HdfsTable extends Table {
    * Otherwise, returns a value >= 1.
    */
   public long getExtrapolatedNumRows(long fileBytes) {
-    if (!BackendConfig.INSTANCE.enableStatsExtrapolation()) return -1;
+    if (!isStatsExtrapolationEnabled()) return -1;
     if (fileBytes == 0) return 0;
     if (fileBytes < 0) return -1;
     if (tableStats_.num_rows < 0 || tableStats_.total_file_bytes <= 0) return -1;
@@ -1962,6 +1968,18 @@ public class HdfsTable extends Table {
   }
 
   /**
+   * Returns true if stats extrapolation is enabled for this table, false otherwise.
+   * Reconciles the Impalad-wide --enable_stats_extrapolation flag and the
+   * TBL_PROP_ENABLE_STATS_EXTRAPOLATION table property
+   */
+  public boolean isStatsExtrapolationEnabled() {
+    org.apache.hadoop.hive.metastore.api.Table msTbl = getMetaStoreTable();
+    String propVal = msTbl.getParameters().get(TBL_PROP_ENABLE_STATS_EXTRAPOLATION);
+    if (propVal == null) return BackendConfig.INSTANCE.isStatsExtrapolationEnabled();
+    return Boolean.parseBoolean(propVal);
+  }
+
+  /**
    * Returns statistics on this table as a tabular result set. Used for the
    * SHOW TABLE STATS statement. The schema of the returned TResultSet is set
    * inside this method.
@@ -1978,7 +1996,7 @@ public class HdfsTable extends Table {
       resultSchema.addToColumns(colDesc);
     }
 
-    boolean statsExtrap = BackendConfig.INSTANCE.enableStatsExtrapolation();
+    boolean statsExtrap = isStatsExtrapolationEnabled();
 
     resultSchema.addToColumns(new TColumn("#Rows", Type.BIGINT.toThrift()));
     if (statsExtrap) {

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 4bcf112..45ad8d6 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -1067,7 +1067,7 @@ public class HdfsScanNode extends ScanNode {
       output.append(getStatsExplainString(detailPrefix));
       output.append("\n");
       String extrapRows = String.valueOf(extrapolatedNumRows_);
-      if (!BackendConfig.INSTANCE.enableStatsExtrapolation()) {
+      if (!tbl_.isStatsExtrapolationEnabled()) {
         extrapRows = "disabled";
       } else if (extrapolatedNumRows_ == -1) {
         extrapRows = "unavailable";

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/fe/src/main/java/org/apache/impala/service/BackendConfig.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index 659e717..48d417a 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -54,7 +54,7 @@ public class BackendConfig {
     return !Strings.isNullOrEmpty(backendCfg_.lineage_event_log_dir);
   }
   public long getIncStatsMaxSize() { return backendCfg_.inc_stats_size_limit_bytes; }
-  public boolean enableStatsExtrapolation() {
+  public boolean isStatsExtrapolationEnabled() {
     return backendCfg_.enable_stats_extrapolation;
   }
   public boolean isAuthToLocalEnabled() {

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index 1c02306..80c6916 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1279,7 +1279,67 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     TBackendGflags gflags = BackendConfig.INSTANCE.getBackendCfg();
     boolean origEnableStatsExtrapolation = gflags.isEnable_stats_extrapolation();
     try {
+      // Setup for testing combinations of extrapolation config options.
+      addTestDb("extrap_config", null);
+      addTestTable("create table extrap_config.tbl_prop_unset (i int)");
+      addTestTable("create table extrap_config.tbl_prop_false (i int) " +
+          "tblproperties('impala.enable.stats.extrapolation'='false')");
+      addTestTable("create table extrap_config.tbl_prop_true (i int) " +
+          "tblproperties('impala.enable.stats.extrapolation'='true')");
+      String stmt = "compute stats %s tablesample system (10)";
+      String err = "COMPUTE STATS TABLESAMPLE requires stats extrapolation";
+
+      // Test --enable_stats_extrapolation=false
+      gflags.setEnable_stats_extrapolation(false);
+      // Table property unset --> Extrapolation disabled
+      AnalysisError(String.format(stmt, "extrap_config.tbl_prop_unset"), err);
+      // Table property false --> Extrapolation disabled
+      AnalysisError(String.format(stmt, "extrap_config.tbl_prop_false"), err);
+      // Table property true --> Extrapolation enabled
+      AnalyzesOk(String.format(stmt, "extrap_config.tbl_prop_true"));
+
+      // Test --enable_stats_extrapolation=true
+      gflags.setEnable_stats_extrapolation(true);
+      // Table property unset --> Extrapolation enabled
+      AnalyzesOk(String.format(stmt, "extrap_config.tbl_prop_unset"));
+      // Table property false --> Extrapolation disabled
+      AnalysisError(String.format(stmt, "extrap_config.tbl_prop_false"), err);
+      // Table property true --> Extrapolation enabled
+      AnalyzesOk(String.format(stmt, "extrap_config.tbl_prop_true"));
+
+      // Test file formats.
       gflags.setEnable_stats_extrapolation(true);
+      checkComputeStatsStmt("compute stats functional.alltypes tablesample system (10)");
+      checkComputeStatsStmt(
+          "compute stats functional.alltypes tablesample system (55) repeatable(1)");
+      AnalysisError("compute stats functional.alltypes tablesample system (101)",
+          "Invalid percent of bytes value '101'. " +
+          "The percent of bytes to sample must be between 0 and 100.");
+      AnalysisError("compute stats functional_kudu.alltypes tablesample system (1)",
+          "TABLESAMPLE is only supported on HDFS tables.");
+      AnalysisError("compute stats functional_hbase.alltypes tablesample system (2)",
+          "TABLESAMPLE is only supported on HDFS tables.");
+      AnalysisError(
+          "compute stats functional.alltypes_datasource tablesample system (3)",
+          "TABLESAMPLE is only supported on HDFS tables.");
+
+      // Test file formats with columns whitelist.
+      gflags.setEnable_stats_extrapolation(true);
+      checkComputeStatsStmt(
+          "compute stats functional.alltypes (int_col, double_col) tablesample " +
+          "system (55) repeatable(1)",
+          Lists.newArrayList("int_col", "double_col"));
+      AnalysisError("compute stats functional.alltypes tablesample system (101)",
+          "Invalid percent of bytes value '101'. " +
+          "The percent of bytes to sample must be between 0 and 100.");
+      AnalysisError("compute stats functional_kudu.alltypes tablesample system (1)",
+          "TABLESAMPLE is only supported on HDFS tables.");
+      AnalysisError("compute stats functional_hbase.alltypes tablesample system (2)",
+          "TABLESAMPLE is only supported on HDFS tables.");
+      AnalysisError(
+          "compute stats functional.alltypes_datasource tablesample system (3)",
+          "TABLESAMPLE is only supported on HDFS tables.");
+
       // Test different COMPUTE_STATS_MIN_SAMPLE_BYTES.
       TQueryOptions queryOpts = new TQueryOptions();
 
@@ -1328,26 +1388,6 @@ public class AnalyzeDDLTest extends FrontendTestBase {
       // changes. Expect a sample between 4 and 6 of the 24 total files.
       Assert.assertTrue(adjustedStmt.getEffectiveSamplingPerc() >= 4.0 / 24);
       Assert.assertTrue(adjustedStmt.getEffectiveSamplingPerc() <= 6.0 / 24);
-      // Checks that whitelisted columns works with tablesample.
-      checkComputeStatsStmt(
-          "compute stats functional.alltypes (int_col, double_col) tablesample " +
-          "system (55) repeatable(1)",
-          Lists.newArrayList("int_col", "double_col"));
-      AnalysisError("compute stats functional.alltypes tablesample system (101)",
-          "Invalid percent of bytes value '101'. " +
-          "The percent of bytes to sample must be between 0 and 100.");
-      AnalysisError("compute stats functional_kudu.alltypes tablesample system (1)",
-          "TABLESAMPLE is only supported on HDFS tables.");
-      AnalysisError("compute stats functional_hbase.alltypes tablesample system (2)",
-          "TABLESAMPLE is only supported on HDFS tables.");
-      AnalysisError(
-          "compute stats functional.alltypes_datasource tablesample system (3)",
-          "TABLESAMPLE is only supported on HDFS tables.");
-
-      gflags.setEnable_stats_extrapolation(false);
-      AnalysisError("compute stats functional.alltypes tablesample system (10)",
-          "COMPUTE STATS TABLESAMPLE requires --enable_stats_extrapolation=true. " +
-          "Stats extrapolation is currently disabled.");
     } finally {
       gflags.setEnable_stats_extrapolation(origEnableStatsExtrapolation);
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/fe/src/test/java/org/apache/impala/planner/StatsExtrapolationTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/StatsExtrapolationTest.java b/fe/src/test/java/org/apache/impala/planner/StatsExtrapolationTest.java
index bee6a32..a0a8566 100644
--- a/fe/src/test/java/org/apache/impala/planner/StatsExtrapolationTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/StatsExtrapolationTest.java
@@ -20,6 +20,7 @@ package org.apache.impala.planner;
 import static org.junit.Assert.assertEquals;
 
 import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.impala.catalog.HdfsTable;
@@ -32,28 +33,36 @@ import org.junit.Test;
 import com.google.common.base.Preconditions;
 
 /**
- * Tests the behavior of stats extrapolation with valid, invalid, and unset stats,
- * as well as extreme values and other edge cases.
+ * Tests the configuration options and behavior of stats extrapolation with valid,
+ * invalid, and unset stats, as well as extreme values and other edge cases.
  */
 public class StatsExtrapolationTest extends FrontendTestBase {
 
   /**
    * Sets the row count and total file size stats in the given table.
    * Unsets the corresponding statistic if a null value is passed.
+   * Preserves existing table properties.
    */
   private void setStats(Table tbl, Long rowCount, Long totalSize) {
-    org.apache.hadoop.hive.metastore.api.Table msTbl =
-        new org.apache.hadoop.hive.metastore.api.Table();
-    msTbl.setParameters(new HashMap<String, String>());
+    org.apache.hadoop.hive.metastore.api.Table msTbl = tbl.getMetaStoreTable();
+    if (msTbl == null) {
+      msTbl = new org.apache.hadoop.hive.metastore.api.Table();
+      msTbl.setParameters(new HashMap<String, String>());
+    }
+    if (msTbl.getParameters() == null) {
+      msTbl.setParameters(new HashMap<String, String>());
+    }
+    Map<String, String> params = msTbl.getParameters();
     if (rowCount != null) {
-      msTbl.getParameters().put(StatsSetupConst.ROW_COUNT,
-          String.valueOf(rowCount));
+      params.put(StatsSetupConst.ROW_COUNT, String.valueOf(rowCount));
+    } else {
+      params.remove(StatsSetupConst.ROW_COUNT);
     }
     if (totalSize != null) {
-      msTbl.getParameters().put(StatsSetupConst.TOTAL_SIZE,
-          String.valueOf(totalSize));
+      params.put(StatsSetupConst.TOTAL_SIZE, String.valueOf(totalSize));
+    } else {
+      params.remove(StatsSetupConst.TOTAL_SIZE);
     }
-    tbl.setMetaStoreTable(msTbl);
     tbl.setTableStats(msTbl);
   }
 
@@ -61,8 +70,8 @@ public class StatsExtrapolationTest extends FrontendTestBase {
       long fileBytes, long expectedExtrapNumRows) {
     Preconditions.checkState(tbl instanceof HdfsTable);
     setStats(tbl, rowCount, totalSize);
-    long actualExrtapNumRows = ((HdfsTable)tbl).getExtrapolatedNumRows(fileBytes);
-    assertEquals(expectedExtrapNumRows, actualExrtapNumRows);
+    long actualExtrapNumRows = ((HdfsTable)tbl).getExtrapolatedNumRows(fileBytes);
+    assertEquals(expectedExtrapNumRows, actualExtrapNumRows);
   }
 
   private void testInvalidStats(Table tbl, Long rowCount, Long totalSize) {
@@ -79,7 +88,7 @@ public class StatsExtrapolationTest extends FrontendTestBase {
     addTestDb("extrap_stats", null);
     Table tbl = addTestTable("create table extrap_stats.t (i int)");
 
-    // Modify/restore the backend config for this test.
+    // Replace/restore the static backend config for this test.
     TBackendGflags gflags = BackendConfig.INSTANCE.getBackendCfg();
     boolean origEnableStatsExtrapolation = gflags.isEnable_stats_extrapolation();
     try {
@@ -134,24 +143,56 @@ public class StatsExtrapolationTest extends FrontendTestBase {
   }
 
   @Test
-  public void TestStatsExtrapolationDisabled() {
-    addTestDb("extrap_stats", null);
-    Table tbl = addTestTable("create table extrap_stats.t (i int)");
-
-    // Modify/restore the backend config for this test.
+  public void TestStatsExtrapolationConfig() {
+    addTestDb("extrap_config", null);
+    Table propUnsetTbl =
+        addTestTable("create table extrap_config.tbl_prop_unset (i int)");
+    Table propFalseTbl =
+        addTestTable("create table extrap_config.tbl_prop_false (i int) " +
+        "tblproperties('impala.enable.stats.extrapolation'='false')");
+    Table propTrueTbl =
+        addTestTable("create table extrap_config.tbl_prop_true (i int) " +
+        "tblproperties('impala.enable.stats.extrapolation'='true')");
+
+    // Replace/restore the static backend config for this test.
     TBackendGflags gflags = BackendConfig.INSTANCE.getBackendCfg();
     boolean origEnableStatsExtrapolation = gflags.isEnable_stats_extrapolation();
     try {
+      // Test --enable_stats_extrapolation=false
       gflags.setEnable_stats_extrapolation(false);
-
-      // Always expect -1 even with legitimate stats.
-      runTest(tbl, 100L, 1000L, 0, -1);
-      runTest(tbl, 100L, 1000L, 100, -1);
-      runTest(tbl, 100L, 1000L, 1000000000, -1);
-      runTest(tbl, 100L, 1000L, Long.MAX_VALUE, -1);
-      runTest(tbl, 100L, 1000L, -100, -1);
+      // Table property unset --> Extrapolation disabled
+      configTestExtrapolationDisabled(propUnsetTbl);
+      // Table property false --> Extrapolation disabled
+      configTestExtrapolationDisabled(propFalseTbl);
+      // Table property true --> Extrapolation enabled
+      configTestExtrapolationEnabled(propTrueTbl);
+
+      // Test --enable_stats_extrapolation=true
+      gflags.setEnable_stats_extrapolation(true);
+      // Table property unset --> Extrapolation enabled
+      configTestExtrapolationEnabled(propUnsetTbl);
+      // Table property false --> Extrapolation disabled
+      configTestExtrapolationDisabled(propFalseTbl);
+      // Table property true --> Extrapolation enabled
+      configTestExtrapolationEnabled(propTrueTbl);
     } finally {
       gflags.setEnable_stats_extrapolation(origEnableStatsExtrapolation);
     }
   }
+
+  private void configTestExtrapolationDisabled(Table tbl) {
+    runTest(tbl, 100L, 1000L, 0, -1);
+    runTest(tbl, 100L, 1000L, 100, -1);
+    runTest(tbl, 100L, 1000L, 1000000000, -1);
+    runTest(tbl, 100L, 1000L, Long.MAX_VALUE, -1);
+    runTest(tbl, 100L, 1000L, -100, -1);
+  }
+
+  private void configTestExtrapolationEnabled(Table tbl) {
+    runTest(tbl, 100L, 1000L, 0, 0);
+    runTest(tbl, 100L, 1000L, 100, 10);
+    runTest(tbl, 100L, 1000L, 1000000000, 100000000);
+    runTest(tbl, 100L, 1000L, Long.MAX_VALUE, 922337203685477632L);
+    runTest(tbl, 100L, 1000L, -100, -1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
index 814a605..7da7baf 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
@@ -1,6 +1,8 @@
 ====
 ---- QUERY
-create table alltypes like functional_parquet.alltypes;
+# This test relies on a deterministic row order so we use "sort by (id)".
+create table alltypes sort by (id) like functional_parquet.alltypes;
+alter table alltypes set tblproperties("impala.enable.stats.extrapolation"="true");
 insert into alltypes partition(year, month)
 select * from functional_parquet.alltypes where year = 2009;
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/tests/custom_cluster/test_stats_extrapolation.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_stats_extrapolation.py b/tests/custom_cluster/test_stats_extrapolation.py
index 42c3820..2910da2 100644
--- a/tests/custom_cluster/test_stats_extrapolation.py
+++ b/tests/custom_cluster/test_stats_extrapolation.py
@@ -15,16 +15,17 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from os import path
+import pytest
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.test_dimensions import (
     create_exec_option_dimension,
     create_single_exec_option_dimension,
     create_uncompressed_text_dimension)
-from tests.util.hdfs_util import NAMENODE
-
 
 class TestStatsExtrapolation(CustomClusterTestSuite):
+  """Minimal end-to-end test for the --enable_stats_extrapolation impalad flag. This test
+  primarly checks that the flag is propagated to the FE. More testing is done in FE unit
+  tests and metadata/test_stats_extrapolation.py."""
 
   @classmethod
   def get_workload(self):
@@ -37,121 +38,21 @@ class TestStatsExtrapolation(CustomClusterTestSuite):
     cls.ImpalaTestMatrix.add_dimension(
         create_uncompressed_text_dimension(cls.get_workload()))
 
-  @CustomClusterTestSuite.with_args(impalad_args=('--enable_stats_extrapolation=true'))
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(impalad_args="--enable_stats_extrapolation=true")
   def test_stats_extrapolation(self, vector, unique_database):
-    vector.get_value('exec_option')['num_nodes'] = 1
-    vector.get_value('exec_option')['explain_level'] = 2
-    self.run_test_case('QueryTest/stats-extrapolation', vector, unique_database)
-
-  @CustomClusterTestSuite.with_args(impalad_args=('--enable_stats_extrapolation=true'))
-  def test_compute_stats_tablesample(self, vector, unique_database):
-    """COMPUTE STATS TABLESAMPLE is inherently non-deterministic due to its use of
-    SAMPLED_NDV() so we test it specially. The goal of this test is to ensure that
-    COMPUTE STATS TABLESAMPLE computes in-the-right-ballpark stats and successfully
-    stores them in the HMS."""
-
-    # Since our test tables are small, set the minimum sample size to 0 to make sure
-    # we exercise the sampling code paths.
-    self.client.execute("set compute_stats_min_sample_size=0")
-
-    # Test partitioned table.
+    # Test row count extrapolation
+    self.client.execute("set explain_level=2")
+    explain_result = self.client.execute("explain select * from functional.alltypes")
+    assert "extrapolated-rows=7300" in " ".join(explain_result.data)
+    # Test COMPUTE STATS TABLESAMPLE
     part_test_tbl = unique_database + ".alltypes"
     self.clone_table("functional.alltypes", part_test_tbl, True, vector)
-    self.__run_sampling_test(part_test_tbl, "", "functional.alltypes", 1, 3)
-    self.__run_sampling_test(part_test_tbl, "", "functional.alltypes", 10, 7)
-    self.__run_sampling_test(part_test_tbl, "", "functional.alltypes", 20, 13)
-    self.__run_sampling_test(part_test_tbl, "", "functional.alltypes", 100, 99)
-
-    # Test unpartitioned table.
-    nopart_test_tbl = unique_database + ".alltypesnopart"
-    self.client.execute("create table {0} as select * from functional.alltypes"\
-      .format(nopart_test_tbl))
-    # Clone to use as a baseline. We run the regular COMPUTE STATS on this table.
-    nopart_test_tbl_exp = unique_database + ".alltypesnopart_exp"
-    self.clone_table(nopart_test_tbl, nopart_test_tbl_exp, False, vector)
-    self.client.execute("compute stats {0}".format(nopart_test_tbl_exp))
-    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_exp, 1, 3)
-    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_exp, 10, 7)
-    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_exp, 20, 13)
-    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_exp, 100, 99)
-
-    # Test empty table.
-    empty_test_tbl = unique_database + ".empty_tbl"
-    self.clone_table("functional.alltypes", empty_test_tbl, False, vector)
-    self.__run_sampling_test(empty_test_tbl, "", empty_test_tbl, 10, 7)
-
-    # Test wide table. Should not crash or error. This takes a few minutes so restrict
-    # to exhaustive.
-    if self.exploration_strategy() == "exhaustive":
-      wide_test_tbl = unique_database + ".wide"
-      self.clone_table("functional.widetable_1000_cols", wide_test_tbl, False, vector)
-      self.client.execute(
-        "compute stats {0} tablesample system(10)".format(wide_test_tbl))
-
-    # Test column subset.
-    column_subset_tbl = unique_database + ".column_subset"
-    columns = "(int_col, string_col)"
-    self.clone_table("functional.alltypes", column_subset_tbl, True, vector)
-    self.__run_sampling_test(column_subset_tbl, columns, "functional.alltypes", 1, 3)
-    self.__run_sampling_test(column_subset_tbl, columns, "functional.alltypes", 10, 7)
-    self.__run_sampling_test(column_subset_tbl, columns, "functional.alltypes", 20, 13)
-    self.__run_sampling_test(column_subset_tbl, columns, "functional.alltypes", 100, 99)
-
-    # Test no columns.
-    no_column_tbl = unique_database + ".no_columns"
-    columns = "()"
-    self.clone_table("functional.alltypes", no_column_tbl, True, vector)
-    self.__run_sampling_test(no_column_tbl, columns, "functional.alltypes", 10, 7)
-
-  def __run_sampling_test(self, tbl, cols, expected_tbl, perc, seed):
-    """Drops stats on 'tbl' and then runs COMPUTE STATS TABLESAMPLE on 'tbl' with the
-    given column restriction clause, sampling percent and random seed. Checks that
-    the resulting table and column stats are reasoanbly close to those of
-    'expected_tbl'."""
-    self.client.execute("drop stats {0}".format(tbl))
-    self.client.execute("compute stats {0}{1} tablesample system ({2}) repeatable ({3})"\
-      .format(tbl, cols, perc, seed))
-    self.__check_table_stats(tbl, expected_tbl)
-    self.__check_column_stats(tbl, expected_tbl)
-
-  def __check_table_stats(self, tbl, expected_tbl):
-    """Checks that the row counts reported in SHOW TABLE STATS on 'tbl' are within 2x
-    of those reported for 'expected_tbl'. Assumes that COMPUTE STATS was previously run
-    on 'expected_table' and that COMPUTE STATS TABLESAMPLE was run on 'tbl'."""
-    actual = self.client.execute("show table stats {0}".format(tbl))
-    expected = self.client.execute("show table stats {0}".format(expected_tbl))
-    assert len(actual.data) == len(expected.data)
-    assert len(actual.schema.fieldSchemas) == len(expected.schema.fieldSchemas)
-    col_names = [fs.name.upper() for fs in actual.schema.fieldSchemas]
-    rows_col_idx = col_names.index("#ROWS")
-    extrap_rows_col_idx = col_names.index("EXTRAP #ROWS")
-    for i in xrange(0, len(actual.data)):
-      act_cols = actual.data[i].split("\t")
-      exp_cols = expected.data[i].split("\t")
-      assert int(exp_cols[rows_col_idx]) >= 0
-      self.appx_equals(\
-        int(act_cols[extrap_rows_col_idx]), int(exp_cols[rows_col_idx]), 2)
-      # Only the table-level row count is stored. The partition row counts
-      # are extrapolated.
-      if act_cols[0] == "Total":
-        self.appx_equals(
-          int(act_cols[rows_col_idx]), int(exp_cols[rows_col_idx]), 2)
-      elif len(actual.data) > 1:
-        # Partition row count is expected to not be set.
-        assert int(act_cols[rows_col_idx]) == -1
-
-  def __check_column_stats(self, tbl, expected_tbl):
-    """Checks that the NDVs in SHOW COLUMNS STATS on 'tbl' are within 2x of those
-    reported for 'expected_tbl'. Assumes that COMPUTE STATS was previously run
-    on 'expected_table' and that COMPUTE STATS TABLESAMPLE was run on 'tbl'."""
-    actual = self.client.execute("show column stats {0}".format(tbl))
-    expected = self.client.execute("show column stats {0}".format(expected_tbl))
-    assert len(actual.data) == len(expected.data)
-    assert len(actual.schema.fieldSchemas) == len(expected.schema.fieldSchemas)
-    col_names = [fs.name.upper() for fs in actual.schema.fieldSchemas]
+    self.client.execute(
+        "compute stats {0} tablesample system (13)".format(part_test_tbl))
+    # Check that column stats were set.
+    col_stats = self.client.execute("show column stats {0}".format(part_test_tbl))
+    col_names = [fs.name.upper() for fs in col_stats.schema.fieldSchemas]
     ndv_col_idx = col_names.index("#DISTINCT VALUES")
-    for i in xrange(0, len(actual.data)):
-      act_cols = actual.data[i].split("\t")
-      exp_cols = expected.data[i].split("\t")
-      assert int(exp_cols[ndv_col_idx]) >= 0
-      self.appx_equals(int(act_cols[ndv_col_idx]), int(exp_cols[ndv_col_idx]), 2)
+    for row in col_stats.data:
+      assert int(row.split("\t")[ndv_col_idx]) >= 0

http://git-wip-us.apache.org/repos/asf/impala/blob/d1816106/tests/metadata/test_stats_extrapolation.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_stats_extrapolation.py b/tests/metadata/test_stats_extrapolation.py
new file mode 100644
index 0000000..61bdb39
--- /dev/null
+++ b/tests/metadata/test_stats_extrapolation.py
@@ -0,0 +1,175 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from os import path
+from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.test_dimensions import (
+    create_exec_option_dimension,
+    create_single_exec_option_dimension,
+    create_uncompressed_text_dimension)
+
+class TestStatsExtrapolation(ImpalaTestSuite):
+  """Test stats extrapolation and compute stats tablesample. Stats extrapolation is
+  enabled via table property and not via the impalad startup flag so these tests can be
+  run as regular tests (non-custom-cluster) and in parallel with other tests."""
+
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestStatsExtrapolation, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(
+        create_uncompressed_text_dimension(cls.get_workload()))
+
+  def test_stats_extrapolation(self, vector, unique_database):
+    vector.get_value('exec_option')['num_nodes'] = 1
+    vector.get_value('exec_option')['explain_level'] = 2
+    self.run_test_case('QueryTest/stats-extrapolation', vector, unique_database)
+
+  def test_compute_stats_tablesample(self, vector, unique_database):
+    """COMPUTE STATS TABLESAMPLE is inherently non-deterministic due to its use of
+    SAMPLED_NDV() so we test it specially. The goal of this test is to ensure that
+    COMPUTE STATS TABLESAMPLE computes in-the-right-ballpark stats and successfully
+    stores them in the HMS."""
+
+    # Since our test tables are small, set the minimum sample size to 0 to make sure
+    # we exercise the sampling code paths.
+    self.client.execute("set compute_stats_min_sample_size=0")
+
+    # Test partitioned table.
+    part_test_tbl = unique_database + ".alltypes"
+    self.clone_table("functional.alltypes", part_test_tbl, True, vector)
+    # Clone to use as a baseline. We run the regular COMPUTE STATS on this table.
+    part_test_tbl_base = unique_database + ".alltypes_base"
+    self.clone_table(part_test_tbl, part_test_tbl_base, True, vector)
+    self.client.execute("compute stats {0}".format(part_test_tbl_base))
+    # Enable stats extrapolation on both tables to match SHOW output.
+    self.__set_extrapolation_tblprop(part_test_tbl)
+    self.__set_extrapolation_tblprop(part_test_tbl_base)
+    self.__run_sampling_test(part_test_tbl, "", part_test_tbl_base, 1, 3)
+    self.__run_sampling_test(part_test_tbl, "", part_test_tbl_base, 10, 7)
+    self.__run_sampling_test(part_test_tbl, "", part_test_tbl_base, 20, 13)
+    self.__run_sampling_test(part_test_tbl, "", part_test_tbl_base, 100, 99)
+
+    # Test unpartitioned table.
+    nopart_test_tbl = unique_database + ".alltypesnopart"
+    self.client.execute("create table {0} as select * from functional.alltypes"\
+      .format(nopart_test_tbl))
+    # Clone to use as a baseline. We run the regular COMPUTE STATS on this table.
+    nopart_test_tbl_base = unique_database + ".alltypesnopart_base"
+    self.clone_table(nopart_test_tbl, nopart_test_tbl_base, False, vector)
+    self.client.execute("compute stats {0}".format(nopart_test_tbl_base))
+    # Enable stats extrapolation on both tables to match SHOW output.
+    self.__set_extrapolation_tblprop(nopart_test_tbl)
+    self.__set_extrapolation_tblprop(nopart_test_tbl_base)
+    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_base, 1, 3)
+    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_base, 10, 7)
+    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_base, 20, 13)
+    self.__run_sampling_test(nopart_test_tbl, "", nopart_test_tbl_base, 100, 99)
+
+    # Test empty table.
+    empty_test_tbl = unique_database + ".empty_tbl"
+    self.clone_table("functional.alltypes", empty_test_tbl, False, vector)
+    self.__set_extrapolation_tblprop(empty_test_tbl)
+    self.__run_sampling_test(empty_test_tbl, "", empty_test_tbl, 10, 7)
+
+    # Test column subset.
+    column_subset_tbl = unique_database + ".column_subset"
+    columns = "(int_col, string_col)"
+    self.clone_table("functional.alltypes", column_subset_tbl, True, vector)
+    self.__set_extrapolation_tblprop(column_subset_tbl)
+    self.__run_sampling_test(column_subset_tbl, columns, part_test_tbl_base, 1, 3)
+    self.__run_sampling_test(column_subset_tbl, columns, part_test_tbl_base, 10, 7)
+    self.__run_sampling_test(column_subset_tbl, columns, part_test_tbl_base, 20, 13)
+    self.__run_sampling_test(column_subset_tbl, columns, part_test_tbl_base, 100, 99)
+
+    # Test no columns.
+    no_column_tbl = unique_database + ".no_columns"
+    columns = "()"
+    self.clone_table("functional.alltypes", no_column_tbl, True, vector)
+    self.__set_extrapolation_tblprop(no_column_tbl)
+    self.__run_sampling_test(no_column_tbl, columns, part_test_tbl_base, 10, 7)
+
+    # Test wide table. Should not crash or error. This takes a few minutes so restrict
+    # to exhaustive.
+    if self.exploration_strategy() == "exhaustive":
+      wide_test_tbl = unique_database + ".wide"
+      self.clone_table("functional.widetable_1000_cols", wide_test_tbl, False, vector)
+      self.__set_extrapolation_tblprop(wide_test_tbl)
+      self.client.execute(
+        "compute stats {0} tablesample system(10)".format(wide_test_tbl))
+
+  def __set_extrapolation_tblprop(self, tbl):
+    """Alters the given table to enable stats extrapolation via tblproperty."""
+    self.client.execute("alter table {0} set "\
+      "tblproperties('impala.enable.stats.extrapolation'='true')".format(tbl))
+
+  def __run_sampling_test(self, tbl, cols, expected_tbl, perc, seed):
+    """Drops stats on 'tbl' and then runs COMPUTE STATS TABLESAMPLE on 'tbl' with the
+    given column restriction clause, sampling percent and random seed. Checks that
+    the resulting table and column stats are reasoanbly close to those of
+    'expected_tbl'."""
+    self.client.execute("drop stats {0}".format(tbl))
+    self.client.execute("compute stats {0}{1} tablesample system ({2}) repeatable ({3})"\
+      .format(tbl, cols, perc, seed))
+    self.__check_table_stats(tbl, expected_tbl)
+    self.__check_column_stats(tbl, expected_tbl)
+
+  def __check_table_stats(self, tbl, expected_tbl):
+    """Checks that the row counts reported in SHOW TABLE STATS on 'tbl' are within 2x
+    of those reported for 'expected_tbl'. Assumes that COMPUTE STATS was previously run
+    on 'expected_table' and that COMPUTE STATS TABLESAMPLE was run on 'tbl'."""
+    actual = self.client.execute("show table stats {0}".format(tbl))
+    expected = self.client.execute("show table stats {0}".format(expected_tbl))
+    assert len(actual.data) == len(expected.data)
+    assert len(actual.schema.fieldSchemas) == len(expected.schema.fieldSchemas)
+    col_names = [fs.name.upper() for fs in actual.schema.fieldSchemas]
+    rows_col_idx = col_names.index("#ROWS")
+    extrap_rows_col_idx = col_names.index("EXTRAP #ROWS")
+    for i in xrange(0, len(actual.data)):
+      act_cols = actual.data[i].split("\t")
+      exp_cols = expected.data[i].split("\t")
+      assert int(exp_cols[rows_col_idx]) >= 0
+      self.appx_equals(\
+        int(act_cols[extrap_rows_col_idx]), int(exp_cols[rows_col_idx]), 2)
+      # Only the table-level row count is stored. The partition row counts
+      # are extrapolated.
+      if act_cols[0] == "Total":
+        self.appx_equals(
+          int(act_cols[rows_col_idx]), int(exp_cols[rows_col_idx]), 2)
+      elif len(actual.data) > 1:
+        # Partition row count is expected to not be set.
+        assert int(act_cols[rows_col_idx]) == -1
+
+  def __check_column_stats(self, tbl, expected_tbl):
+    """Checks that the NDVs in SHOW COLUMNS STATS on 'tbl' are within 2x of those
+    reported for 'expected_tbl'. Assumes that COMPUTE STATS was previously run
+    on 'expected_table' and that COMPUTE STATS TABLESAMPLE was run on 'tbl'."""
+    actual = self.client.execute("show column stats {0}".format(tbl))
+    expected = self.client.execute("show column stats {0}".format(expected_tbl))
+    assert len(actual.data) == len(expected.data)
+    assert len(actual.schema.fieldSchemas) == len(expected.schema.fieldSchemas)
+    col_names = [fs.name.upper() for fs in actual.schema.fieldSchemas]
+    ndv_col_idx = col_names.index("#DISTINCT VALUES")
+    for i in xrange(0, len(actual.data)):
+      act_cols = actual.data[i].split("\t")
+      exp_cols = expected.data[i].split("\t")
+      assert int(exp_cols[ndv_col_idx]) >= 0
+      self.appx_equals(int(act_cols[ndv_col_idx]), int(exp_cols[ndv_col_idx]), 2)


[11/21] impala git commit: IMPALA-6456: Add flags to configure rpc_negotiation_timeout_ms and negotiation thread count in KRPC

Posted by ta...@apache.org.
IMPALA-6456: Add flags to configure rpc_negotiation_timeout_ms and negotiation thread count in KRPC

With the fix for KUDU-2228, the FLAGS_rpc_negotiation_timeout_ms was
retired in KRPC.

This patch introduces a flag to be able to configure that from the
Impala side (FLAGS_rpc_negotiation_timeout_ms).

It also introduces a flag to configure the negotiation
thread count (FLAGS_rpc_negotiation_thread_count).

Added a test to verify that setting FLAGS_rpc_negotiation_timeout_ms
to 0 causes negotiation failures. We unfortunately can't write a test
to check the same for FLAGS_rpc_negotiation_thread_count due to
DCHECKS present in the code.

Change-Id: I108d700e7eac04b678e21a3a920aac81ba8eede5
Reviewed-on: http://gerrit.cloudera.org:8080/9186
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 83f9650186a2be5271142a00af6bde5128a16102
Parents: 31f3868
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Thu Feb 1 16:06:57 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/rpc/rpc-mgr-test.cc | 22 ++++++++++++++++++++++
 be/src/rpc/rpc-mgr.cc      |  6 ++++++
 2 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/83f96501/be/src/rpc/rpc-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-test.cc b/be/src/rpc/rpc-mgr-test.cc
index 4c4b100..cd24672 100644
--- a/be/src/rpc/rpc-mgr-test.cc
+++ b/be/src/rpc/rpc-mgr-test.cc
@@ -24,6 +24,7 @@ using kudu::MonoDelta;
 
 DECLARE_int32(num_reactor_threads);
 DECLARE_int32(num_acceptor_threads);
+DECLARE_int32(rpc_negotiation_timeout_ms);
 DECLARE_string(hostname);
 
 namespace impala {
@@ -230,6 +231,27 @@ TEST_F(RpcMgrTest, AsyncCall) {
   }
 }
 
+// Run a test with the negotiation timeout as 0 ms and ensure that connection
+// establishment fails.
+// This is to verify that FLAGS_rpc_negotiation_timeout_ms is actually effective.
+TEST_F(RpcMgrTest, NegotiationTimeout) {
+  // Set negotiation timeout to 0 milliseconds.
+  auto s = ScopedFlagSetter<int32_t>::Make(&FLAGS_rpc_negotiation_timeout_ms, 0);
+
+  RpcMgr secondary_rpc_mgr(IsInternalTlsConfigured());
+  TNetworkAddress secondary_krpc_address;
+  IpAddr ip;
+  ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+
+  int32_t secondary_service_port = FindUnusedEphemeralPort(nullptr);
+  secondary_krpc_address = MakeNetworkAddress(ip, secondary_service_port);
+
+  ASSERT_OK(secondary_rpc_mgr.Init());
+  ASSERT_FALSE(RunMultipleServicesTestTemplate(
+      this, &secondary_rpc_mgr, secondary_krpc_address).ok());
+  secondary_rpc_mgr.Shutdown();
+}
+
 } // namespace impala
 
 int main(int argc, char** argv) {

http://git-wip-us.apache.org/repos/asf/impala/blob/83f96501/be/src/rpc/rpc-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr.cc b/be/src/rpc/rpc-mgr.cc
index 7adde36..44ecc02 100644
--- a/be/src/rpc/rpc-mgr.cc
+++ b/be/src/rpc/rpc-mgr.cc
@@ -64,6 +64,10 @@ DEFINE_int32(num_reactor_threads, 0,
     "default value 0, it will be set to number of CPU cores.");
 DEFINE_int32(rpc_retry_interval_ms, 5,
     "Time in millisecond of waiting before retrying an RPC when remote is busy");
+DEFINE_int32(rpc_negotiation_timeout_ms, 60000,
+    "Time in milliseconds of waiting for a negotiation to complete before timing out.");
+DEFINE_int32(rpc_negotiation_thread_count, 4,
+    "Maximum number of threads dedicated to handling RPC connection negotiations.");
 
 namespace impala {
 
@@ -77,6 +81,8 @@ Status RpcMgr::Init() {
   int num_reactor_threads =
       FLAGS_num_reactor_threads > 0 ? FLAGS_num_reactor_threads : CpuInfo::num_cores();
   bld.set_num_reactors(num_reactor_threads).set_metric_entity(entity);
+  bld.set_rpc_negotiation_timeout_ms(FLAGS_rpc_negotiation_timeout_ms);
+  bld.set_max_negotiation_threads(max(1, FLAGS_rpc_negotiation_thread_count));
 
   // Disable idle connection detection by setting keepalive_time to -1. Idle connections
   // tend to be closed and re-opened around the same time, which may lead to negotiation


[14/21] impala git commit: IMPALA-6219: Use AES-GCM for spill-to-disk encryption

Posted by ta...@apache.org.
IMPALA-6219: Use AES-GCM for spill-to-disk encryption

AES-GCM can be very fast(~10 times faster than CFB+SHA256), but it
requires an instruction that Impala can currently run without (CLMUL).
In order to be fast, we dispatch to GCM mode at run-time based on the
CPU and OpenSSL version.

Testing:
run runtime tmp-file-mgr-test, openssl-util-test, buffer-pool-test
and buffered-tuple-stream-test.
add two cases GcmIntegrity & EncryptoArbitraryLength for
openssl-util-test

Change-Id: I1ea87b82a8897ee8bfa187715ac1c52883790d24
Reviewed-on: http://gerrit.cloudera.org:8080/9032
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: e7df3b2fa389cc0829d1575e5b66d193e28553a2
Parents: d84657b
Author: Xianda Ke <ke...@gmail.com>
Authored: Tue Jan 16 16:23:28 2018 +0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/runtime/tmp-file-mgr.cc   | 15 ++++--
 be/src/util/cpu-info.cc          | 13 ++---
 be/src/util/cpu-info.h           | 13 ++---
 be/src/util/openssl-util-test.cc | 95 +++++++++++++++++++++++-----------
 be/src/util/openssl-util.cc      | 96 ++++++++++++++++++++++++++++++++---
 be/src/util/openssl-util.h       | 70 +++++++++++++++++--------
 6 files changed, 228 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e7df3b2f/be/src/runtime/tmp-file-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.cc b/be/src/runtime/tmp-file-mgr.cc
index d35d302..3807670 100644
--- a/be/src/runtime/tmp-file-mgr.cc
+++ b/be/src/runtime/tmp-file-mgr.cc
@@ -612,19 +612,26 @@ void TmpFileMgr::WriteHandle::WaitForWrite() {
 Status TmpFileMgr::WriteHandle::EncryptAndHash(MemRange buffer) {
   DCHECK(FLAGS_disk_spill_encryption);
   SCOPED_TIMER(encryption_timer_);
-  // Since we're using AES-CTR/AES-CFB mode, we must take care not to reuse a
+  // Since we're using GCM/CTR/CFB mode, we must take care not to reuse a
   // key/IV pair. Regenerate a new key and IV for every data buffer we write.
   key_.InitializeRandom();
   RETURN_IF_ERROR(key_.Encrypt(buffer.data(), buffer.len(), buffer.data()));
-  hash_.Compute(buffer.data(), buffer.len());
+
+  if (!key_.IsGcmMode()) {
+    hash_.Compute(buffer.data(), buffer.len());
+  }
   return Status::OK();
 }
 
 Status TmpFileMgr::WriteHandle::CheckHashAndDecrypt(MemRange buffer) {
   DCHECK(FLAGS_disk_spill_encryption);
   SCOPED_TIMER(encryption_timer_);
-  if (!hash_.Verify(buffer.data(), buffer.len())) {
-    return Status("Block verification failure");
+
+  // GCM mode will verify the integrity by itself
+  if (!key_.IsGcmMode()) {
+    if (!hash_.Verify(buffer.data(), buffer.len())) {
+      return Status("Block verification failure");
+    }
   }
   return key_.Decrypt(buffer.data(), buffer.len(), buffer.data());
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e7df3b2f/be/src/util/cpu-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index a32571e..1e3fcde 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -85,12 +85,13 @@ static struct {
   int64_t flag;
 } flag_mappings[] =
 {
-  { "ssse3",  CpuInfo::SSSE3 },
-  { "sse4_1", CpuInfo::SSE4_1 },
-  { "sse4_2", CpuInfo::SSE4_2 },
-  { "popcnt", CpuInfo::POPCNT },
-  { "avx",    CpuInfo::AVX },
-  { "avx2",   CpuInfo::AVX2 },
+  { "ssse3",     CpuInfo::SSSE3 },
+  { "sse4_1",    CpuInfo::SSE4_1 },
+  { "sse4_2",    CpuInfo::SSE4_2 },
+  { "popcnt",    CpuInfo::POPCNT },
+  { "avx",       CpuInfo::AVX },
+  { "avx2",      CpuInfo::AVX2 },
+  { "pclmuldqd", CpuInfo::PCLMULQDQ }
 };
 static const long num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/e7df3b2f/be/src/util/cpu-info.h
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.h b/be/src/util/cpu-info.h
index 38d6782..e60babc 100644
--- a/be/src/util/cpu-info.h
+++ b/be/src/util/cpu-info.h
@@ -34,12 +34,13 @@ namespace impala {
 /// /sys/devices)
 class CpuInfo {
  public:
-  static const int64_t SSSE3   = (1 << 1);
-  static const int64_t SSE4_1  = (1 << 2);
-  static const int64_t SSE4_2  = (1 << 3);
-  static const int64_t POPCNT  = (1 << 4);
-  static const int64_t AVX     = (1 << 5);
-  static const int64_t AVX2    = (1 << 6);
+  static const int64_t SSSE3     = (1 << 1);
+  static const int64_t SSE4_1    = (1 << 2);
+  static const int64_t SSE4_2    = (1 << 3);
+  static const int64_t POPCNT    = (1 << 4);
+  static const int64_t AVX       = (1 << 5);
+  static const int64_t AVX2      = (1 << 6);
+  static const int64_t PCLMULQDQ = (1 << 7);
 
   /// Cache enums for L1 (data), L2 and L3
   enum CacheLevel {

http://git-wip-us.apache.org/repos/asf/impala/blob/e7df3b2f/be/src/util/openssl-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util-test.cc b/be/src/util/openssl-util-test.cc
index 8d98b0d..76f65a5 100644
--- a/be/src/util/openssl-util-test.cc
+++ b/be/src/util/openssl-util-test.cc
@@ -44,6 +44,41 @@ class OpenSSLUtilTest : public ::testing::Test {
     }
   }
 
+  /// Fill arbitrary-length buffer with random bytes
+  void GenerateRandomBytes(uint8_t* data, int64_t len) {
+    DCHECK_GE(len, 0);
+    for (int64_t i = 0; i < len; i++) {
+      data[i] = uniform_int_distribution<uint8_t>(0, UINT8_MAX)(rng_);
+    }
+  }
+
+  void TestEncryptionDecryption(const int64_t buffer_size) {
+    vector<uint8_t> original(buffer_size);
+    vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
+    if (buffer_size % 8 == 0) {
+      GenerateRandomData(original.data(), buffer_size);
+    } else {
+      GenerateRandomBytes(original.data(), buffer_size);
+    }
+
+    // Check all the modes
+    AES_CIPHER_MODE modes[] = {AES_256_GCM, AES_256_CTR, AES_256_CFB};
+    for (auto m : modes) {
+      memcpy(scratch.data(), original.data(), buffer_size);
+
+      EncryptionKey key;
+      key.InitializeRandom();
+      key.SetCipherMode(m);
+
+      ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
+      // Check that encryption did something
+      ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
+      ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
+      // Check that we get the original data back.
+      ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
+    }
+  }
+
   mt19937_64 rng_;
 };
 
@@ -57,7 +92,7 @@ TEST_F(OpenSSLUtilTest, Encryption) {
   GenerateRandomData(original.data(), buffer_size);
 
   // Check both CTR & CFB
-  AES_CIPHER_MODE modes[] = {AES_256_CTR, AES_256_CFB};
+  AES_CIPHER_MODE modes[] = {AES_256_GCM, AES_256_CTR, AES_256_CFB};
   for (auto m : modes) {
     // Iterate multiple times to ensure that key regeneration works correctly.
     EncryptionKey key;
@@ -85,44 +120,42 @@ TEST_F(OpenSSLUtilTest, Encryption) {
 /// Test that encryption and decryption work in-place.
 TEST_F(OpenSSLUtilTest, EncryptInPlace) {
   const int buffer_size = 1024 * 1024;
-  vector<uint8_t> original(buffer_size);
-  vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
-
-  EncryptionKey key;
-  // Check both CTR & CFB
-  AES_CIPHER_MODE modes[] = {AES_256_CTR, AES_256_CFB};
-  for (auto m : modes) {
-    GenerateRandomData(original.data(), buffer_size);
-    memcpy(scratch.data(), original.data(), buffer_size);
-
-    key.InitializeRandom();
-    key.SetCipherMode(m);
-
-    ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
-    // Check that encryption did something
-    ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
-    ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
-    // Check that we get the original data back.
-    ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
-  }
+  TestEncryptionDecryption(buffer_size);
 }
 
 /// Test that encryption works with buffer lengths that don't fit in a 32-bit integer.
 TEST_F(OpenSSLUtilTest, EncryptInPlaceHugeBuffer) {
   const int64_t buffer_size = 3 * 1024L * 1024L * 1024L;
-  vector<uint8_t> original(buffer_size);
-  vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
-  GenerateRandomData(original.data(), buffer_size);
-  memcpy(scratch.data(), original.data(), buffer_size);
+  TestEncryptionDecryption(buffer_size);
+}
+
+/// Test that encryption works with arbitrary-length buffer
+TEST_F(OpenSSLUtilTest, EncryptArbitraryLength) {
+  std::uniform_int_distribution<uint64_t> dis(0, 1024 * 1024);
+  const int buffer_size = dis(rng_);
+  TestEncryptionDecryption(buffer_size);
+}
+
+/// Test integrity in GCM mode
+TEST_F(OpenSSLUtilTest, GcmIntegrity) {
+  const int buffer_size = 1024 * 1024;
+  vector<uint8_t> buffer(buffer_size);
 
   EncryptionKey key;
   key.InitializeRandom();
-  ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
-  // Check that encryption did something
-  ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
-  ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
-  // Check that we get the original data back.
-  ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
+  key.SetCipherMode(AES_256_GCM);
+
+  // Even it has been set as GCM mode, it may fall back to other modes.
+  // Check if GCM mode is supported at runtime.
+  if (key.IsGcmMode()) {
+    GenerateRandomData(buffer.data(), buffer_size);
+    ASSERT_OK(key.Encrypt(buffer.data(), buffer_size, buffer.data()));
+
+    // tamper the data
+    ++buffer[0];
+    Status s = key.Decrypt(buffer.data(), buffer_size, buffer.data());
+    EXPECT_STR_CONTAINS(s.GetDetail(), "EVP_DecryptFinal");
+  }
 }
 
 /// Test basic integrity hash functionality.

http://git-wip-us.apache.org/repos/asf/impala/blob/e7df3b2f/be/src/util/openssl-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.cc b/be/src/util/openssl-util.cc
index 69dc676..ffb47eb 100644
--- a/be/src/util/openssl-util.cc
+++ b/be/src/util/openssl-util.cc
@@ -20,6 +20,7 @@
 #include <limits.h>
 #include <sstream>
 
+#include <glog/logging.h>
 #include <openssl/err.h>
 #include <openssl/evp.h>
 #include <openssl/rand.h>
@@ -30,6 +31,7 @@
 #include "gutil/strings/substitute.h"
 
 #include "common/names.h"
+#include "cpu-info.h"
 
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_server_certificate);
@@ -107,19 +109,20 @@ void EncryptionKey::InitializeRandom() {
   }
   RAND_bytes(key_, sizeof(key_));
   RAND_bytes(iv_, sizeof(iv_));
+  memset(gcm_tag_, 0, sizeof(gcm_tag_));
   initialized_ = true;
 }
 
-Status EncryptionKey::Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const {
+Status EncryptionKey::Encrypt(const uint8_t* data, int64_t len, uint8_t* out) {
   return EncryptInternal(true, data, len, out);
 }
 
-Status EncryptionKey::Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const {
+Status EncryptionKey::Decrypt(const uint8_t* data, int64_t len, uint8_t* out) {
   return EncryptInternal(false, data, len, out);
 }
 
 Status EncryptionKey::EncryptInternal(
-    bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) const {
+    bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) {
   DCHECK(initialized_);
   DCHECK_GE(len, 0);
   // Create and initialize the context for encryption
@@ -127,6 +130,10 @@ Status EncryptionKey::EncryptInternal(
   EVP_CIPHER_CTX_init(&ctx);
   EVP_CIPHER_CTX_set_padding(&ctx, 0);
 
+  if (IsGcmMode()) {
+    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_SET_IVLEN, AES_BLOCK_SIZE, NULL);
+  }
+
   // Start encryption/decryption.  We use a 256-bit AES key, and the cipher block mode
   // is either CTR or CFB(stream cipher), both of which support arbitrary length
   // ciphertexts - it doesn't have to be a multiple of 16 bytes. Additionally, CTR
@@ -157,6 +164,11 @@ Status EncryptionKey::EncryptInternal(
     offset += in_len;
   }
 
+  if (IsGcmMode() && !encrypt) {
+    // Set expected tag value
+    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_SET_TAG, AES_BLOCK_SIZE, gcm_tag_);
+  }
+
   // Finalize encryption or decryption.
   int final_out_len;
   success = encrypt ? EVP_EncryptFinal_ex(&ctx, out + offset, &final_out_len) :
@@ -164,21 +176,93 @@ Status EncryptionKey::EncryptInternal(
   if (success != 1) {
     return OpenSSLErr(encrypt ? "EVP_EncryptFinal" : "EVP_DecryptFinal");
   }
-  // Again safe due to CTR/CFB with no padding
+
+  if (IsGcmMode() && encrypt) {
+    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_GET_TAG, AES_BLOCK_SIZE, gcm_tag_);
+  }
+
+  // Again safe due to GCM/CTR/CFB with no padding
   DCHECK_EQ(final_out_len, 0);
   return Status::OK();
 }
 
+/// OpenSSL 1.0.1d
+#define OPENSSL_VERSION_1_0_1D 0x1000104fL
+
+/// If not defined at compile time, define them manually
+/// see: openssl/evp.h
+#ifndef EVP_CIPH_GCM_MODE
+#define EVP_CTRL_GCM_SET_IVLEN 0x9
+#define EVP_CTRL_GCM_GET_TAG 0x10
+#define EVP_CTRL_GCM_SET_TAG 0x11
+#endif
+
 extern "C" {
 ATTRIBUTE_WEAK
 const EVP_CIPHER* EVP_aes_256_ctr();
+
+ATTRIBUTE_WEAK
+const EVP_CIPHER* EVP_aes_256_gcm();
 }
 
 const EVP_CIPHER* EncryptionKey::GetCipher() const {
   // use weak symbol to avoid compiling error on OpenSSL 1.0.0 environment
-  if (mode_ == AES_256_CTR && EVP_aes_256_ctr) return EVP_aes_256_ctr();
+  if (mode_ == AES_256_CTR) return EVP_aes_256_ctr();
+  if (mode_ == AES_256_GCM) return EVP_aes_256_gcm();
 
-  // otherwise, fallback to CFB mode
   return EVP_aes_256_cfb();
 }
+
+void EncryptionKey::SetCipherMode(AES_CIPHER_MODE m) {
+  mode_ = m;
+
+  if (!IsModeSupported(m)) {
+    mode_ = GetSupportedDefaultMode();
+    LOG(WARNING) << Substitute("$0 is not supported, fall back to $1.",
+        ModeToString(m), ModeToString(mode_));
+  }
+}
+
+bool EncryptionKey::IsModeSupported(AES_CIPHER_MODE m) const {
+  switch (m) {
+    case AES_256_GCM:
+      // It becomes a bit tricky for GCM mode, because GCM mode is enabled since
+      // OpenSSL 1.0.1, but the tag validation only works since 1.0.1d. We have
+      // to make sure that OpenSSL version >= 1.0.1d for GCM. So we need
+      // SSLeay(). Note that SSLeay() may return the compiling version on
+      // certain platforms if it was built against an older version(see:
+      // IMPALA-6418). In this case, it will return false, and EncryptionKey
+      // will try to fall back to CTR mode, so it is not ideal but is OK to use
+      // SSLeay() for GCM mode here since in the worst case, we will be using
+      // AES_256_CTR in a system that supports AES_256_GCM.
+      return (CpuInfo::IsSupported(CpuInfo::PCLMULQDQ)
+          && SSLeay() >= OPENSSL_VERSION_1_0_1D && EVP_aes_256_gcm);
+
+    case AES_256_CTR:
+      // If TLS1.2 is supported, then we're on a verison of OpenSSL that
+      // supports AES-256-CTR.
+      return (MaxSupportedTlsVersion() >= TLS1_2_VERSION && EVP_aes_256_ctr);
+
+    case AES_256_CFB:
+      return true;
+
+    default:
+      return false;
+  }
+}
+
+AES_CIPHER_MODE EncryptionKey::GetSupportedDefaultMode() const {
+  if (IsModeSupported(AES_256_GCM)) return AES_256_GCM;
+  if (IsModeSupported(AES_256_CTR)) return AES_256_CTR;
+  return AES_256_CFB;
+}
+
+const string EncryptionKey::ModeToString(AES_CIPHER_MODE m) const {
+  switch(m) {
+    case AES_256_GCM: return "AES-GCM";
+    case AES_256_CTR: return "AES-CTR";
+    case AES_256_CFB: return "AES-CFB";
+  }
+  return "Unknown mode";
+}
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/e7df3b2f/be/src/util/openssl-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.h b/be/src/util/openssl-util.h
index 7b1b28e..ef53425 100644
--- a/be/src/util/openssl-util.h
+++ b/be/src/util/openssl-util.h
@@ -60,9 +60,9 @@ bool IsExternalTlsConfigured();
 void SeedOpenSSLRNG();
 
 enum AES_CIPHER_MODE {
-  AES_256_CTR,
   AES_256_CFB,
-  AES_256_GCM // not supported now.
+  AES_256_CTR,
+  AES_256_GCM
 };
 
 /// The hash of a data buffer used for checking integrity. A SHA256 hash is used
@@ -83,43 +83,56 @@ class IntegrityHash {
 /// The key and initialization vector (IV) required to encrypt and decrypt a buffer of
 /// data. This should be regenerated for each buffer of data.
 ///
-/// We use AES with a 256-bit key and CTR/CFB cipher block mode, which gives us a stream
-/// cipher that can support arbitrary-length ciphertexts. If OpenSSL version at runtime
-/// is 1.0.1 or above, CTR mode is used, otherwise CFB mode is used. The IV is used as
+/// We use AES with a 256-bit key and GCM/CTR/CFB cipher block mode, which gives us a
+/// stream cipher that can support arbitrary-length ciphertexts. The mode is chosen
+/// depends on the OpenSSL version & the hardware support at runtime. The IV is used as
 /// an input to the cipher as the "block to supply before the first block of plaintext".
 /// This is required because all ciphers (except the weak ECB) are built such that each
 /// block depends on the output from the previous block. Since the first block doesn't
 /// have a previous block, we supply this IV. Think of it  as starting off the chain of
 /// encryption.
+///
+/// Notes for GCM:
+/// (1) GCM mode was supported since OpenSSL 1.0.1, however the tag verification
+/// in decryption was only supported since OpenSSL 1.0.1d.
+/// (2) The plaintext and the Additional Authenticated Data(AAD) are the two
+/// categories of data that GCM protects. GCM protects the authenticity of the
+/// plaintext and the AAD, and GCM also protects the confidentiality of the
+/// plaintext. The AAD itself is not required or won't change the security.
+/// In our case(Spill to Disk), we just ignore the AAD.
+
 class EncryptionKey {
  public:
-  EncryptionKey() : initialized_(false) {
-    // If TLS1.2 is supported, then we're on a verison of OpenSSL that supports
-    // AES-256-CTR.
-    mode_ = MaxSupportedTlsVersion() < TLS1_2_VERSION ? AES_256_CFB : AES_256_CTR;
-  }
-
-  /// Initialize a key for temporary use with randomly generated data. Reinitializes with
-  /// new random values if the key was already initialized. We use AES-CTR/AES-CFB mode
-  /// so key/IV pairs should not be reused. This function automatically reseeds the RNG
-  /// periodically, so callers do not need to do it.
+  EncryptionKey() : initialized_(false) { mode_ = GetSupportedDefaultMode(); }
+
+  /// Initializes a key for temporary use with randomly generated data, and clears the
+  /// tag for GCM mode. Reinitializes with new random values if the key was already
+  /// initialized. We use AES-GCM/AES-CTR/AES-CFB mode so key/IV pairs should not be
+  /// reused. This function automatically reseeds the RNG periodically, so callers do
+  /// not need to do it.
   void InitializeRandom();
 
   /// Encrypts a buffer of input data 'data' of length 'len' into an output buffer 'out'.
   /// Exactly 'len' bytes will be written to 'out'. This key must be initialized before
   /// calling. Operates in-place if 'in' == 'out', otherwise the buffers must not overlap.
-  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
+  /// For GCM mode, the hash tag will be kept inside(gcm_tag_ variable).
+  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) WARN_UNUSED_RESULT;
 
   /// Decrypts a buffer of input data 'data' of length 'len' that was encrypted with this
   /// key into an output buffer 'out'. Exactly 'len' bytes will be written to 'out'.
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
-  /// otherwise the buffers must not overlap.
-  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
+  /// otherwise the buffers must not overlap. For GCM mode, the hash tag, which is
+  /// computed during encryption, will be used for intgerity verification.
+  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) WARN_UNUSED_RESULT;
 
   /// Specify a cipher mode. Currently used only for testing but maybe in future we
   /// can provide a configuration option for the end user who can choose a preferred
   /// mode(GCM, CTR, CFB...) based on their software/hardware environment.
-  void SetCipherMode(AES_CIPHER_MODE m) { mode_ = m; }
+  /// If not supported, fall back to the supported mode at runtime.
+  void SetCipherMode(AES_CIPHER_MODE m);
+
+  /// If is GCM mode at runtime
+  bool IsGcmMode() const { return mode_ == AES_256_GCM; }
 
  private:
   /// Helper method that encrypts/decrypts if 'encrypt' is true/false respectively.
@@ -128,13 +141,25 @@ class EncryptionKey {
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
   /// otherwise the buffers must not overlap.
   Status EncryptInternal(bool encrypt, const uint8_t* data, int64_t len,
-      uint8_t* out) const WARN_UNUSED_RESULT;
+      uint8_t* out) WARN_UNUSED_RESULT;
+
+  /// Check if mode m is supported at runtime
+  bool IsModeSupported(AES_CIPHER_MODE m) const;
+
+  /// Returns the a default mode which is supported at runtime. If GCM mode
+  /// is supported, return AES_256_GCM as the default. If GCM is not supported,
+  /// but CTR is still supported, return AES_256_CTR. When both GCM and
+  /// CTR modes are not supported, return AES_256_CFB.
+  AES_CIPHER_MODE GetSupportedDefaultMode() const;
+
+  /// Converts mode type to string.
+  const string ModeToString(AES_CIPHER_MODE m) const;
 
   /// Track whether this key has been initialized, to avoid accidentally using
   /// uninitialized keys.
   bool initialized_;
 
-  /// return a EVP_CIPHER according to cipher mode at runtime
+  /// Returns a EVP_CIPHER according to cipher mode at runtime
   const EVP_CIPHER* GetCipher() const;
 
   /// An AES 256-bit key.
@@ -143,6 +168,9 @@ class EncryptionKey {
   /// An initialization vector to feed as the first block to AES.
   uint8_t iv_[AES_BLOCK_SIZE];
 
+  /// Tag for GCM mode
+  uint8_t gcm_tag_[AES_BLOCK_SIZE];
+
   /// Cipher Mode
   AES_CIPHER_MODE mode_;
 };


[06/21] impala git commit: IMPALA-5990: End-to-end compression of metadata

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java b/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
index 4bb6b65..79960e4 100644
--- a/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/ImpaladCatalog.java
@@ -17,13 +17,15 @@
 
 package org.apache.impala.catalog;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.common.ImpalaException;
+import org.apache.impala.common.Pair;
+import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TDataSource;
@@ -36,8 +38,10 @@ import org.apache.impala.thrift.TUniqueId;
 import org.apache.impala.thrift.TUpdateCatalogCacheRequest;
 import org.apache.impala.thrift.TUpdateCatalogCacheResponse;
 import org.apache.impala.util.PatternMatcher;
+import org.apache.impala.util.TByteBuffer;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
 
 import com.google.common.base.Preconditions;
 
@@ -114,12 +118,29 @@ public class ImpaladCatalog extends Catalog {
   }
 
   /**
+   * Update the catalog service Id. Trigger a full update if the service ID changes.
+   */
+  private void setCatalogServiceId(TUniqueId catalog_service_id) throws CatalogException {
+    // Check for changes in the catalog service ID.
+    if (!catalogServiceId_.equals(catalog_service_id)) {
+      boolean firstRun = catalogServiceId_.equals(INITIAL_CATALOG_SERVICE_ID);
+      catalogServiceId_ = catalog_service_id;
+      if (!firstRun) {
+        // Throw an exception which will trigger a full topic update request.
+        throw new CatalogException("Detected catalog service ID change. Aborting " +
+            "updateCatalog()");
+      }
+    }
+  }
+
+  /**
    * Updates the internal Catalog based on the given TCatalogUpdateReq.
    * This method:
-   * 1) Updates all top level objects (such as databases and roles).
-   * 2) Updates all objects that depend on top level objects (such as functions, tables,
+   * 1) Calls NativeGetNextCatalogObjectUpdate() to get all the updates from the backend.
+   * 2) Updates all top level objects (such as databases and roles).
+   * 3) Updates all objects that depend on top level objects (such as functions, tables,
    *    privileges).
-   * 3) Removes all dropped catalog objects.
+   * 4) Removes all dropped catalog objects.
    *
    * This method is called once per statestore heartbeat and is guaranteed the same
    * object will not be in both the "updated" list and the "removed" list (it is
@@ -132,60 +153,58 @@ public class ImpaladCatalog extends Catalog {
    * protected.
    */
   public synchronized TUpdateCatalogCacheResponse updateCatalog(
-    TUpdateCatalogCacheRequest req) throws CatalogException {
-    // Check for changes in the catalog service ID.
-    if (!catalogServiceId_.equals(req.getCatalog_service_id())) {
-      boolean firstRun = catalogServiceId_.equals(INITIAL_CATALOG_SERVICE_ID);
-      catalogServiceId_ = req.getCatalog_service_id();
-      if (!firstRun) {
-        // Throw an exception which will trigger a full topic update request.
-        throw new CatalogException("Detected catalog service ID change. Aborting " +
-            "updateCatalog()");
+    TUpdateCatalogCacheRequest req) throws CatalogException, TException {
+    // For updates from catalog op results, the service ID is set in the request.
+    if (req.isSetCatalog_service_id()) setCatalogServiceId(req.catalog_service_id);
+    ArrayDeque<TCatalogObject> updatedObjects = new ArrayDeque<>();
+    ArrayDeque<TCatalogObject> deletedObjects = new ArrayDeque<>();
+    long newCatalogVersion = lastSyncedCatalogVersion_;
+    Pair<Boolean, ByteBuffer> update;
+    while ((update = FeSupport.NativeGetNextCatalogObjectUpdate(req.native_iterator_ptr))
+        != null) {
+      TCatalogObject obj = new TCatalogObject();
+      obj.read(new TBinaryProtocol(new TByteBuffer(update.second)));
+      String key = Catalog.toCatalogObjectKey(obj);
+      int len = update.second.capacity();
+      if (len > 100 * 1024 * 1024 /* 100MB */) {
+        LOG.info("Received large catalog object(>100mb): " + key + " is " + len +
+            "bytes");
       }
-    }
-
-    // Process updates to top level objects first because they don't depend on any other
-    // objects already existing in the catalog.
-    for (TCatalogObject catalogObject: req.getUpdated_objects()) {
-      if (isTopLevelCatalogObject(catalogObject)) {
-        Preconditions.checkState(catalogObject.getType() != TCatalogObjectType.CATALOG);
-        try {
-          addCatalogObject(catalogObject);
-        } catch (Exception e) {
-          LOG.error("Error adding catalog object: " + e.getMessage(), e);
-        }
+      if (LOG.isTraceEnabled()) {
+        LOG.trace((update.first ? "Deleting " : "Adding ") + "item: " + key + " version: "
+            + obj.catalog_version + " of size: " + len);
       }
-    }
-
-    // Process updates to dependent objects next. Since the top level objects were already
-    // processed, we are guaranteed that the top level objects that the dependent objects
-    // depend on exist in the catalog.
-    long newCatalogVersion = lastSyncedCatalogVersion_;
-    for (TCatalogObject catalogObject: req.getUpdated_objects()) {
-      if (catalogObject.getType() == TCatalogObjectType.CATALOG) {
-        newCatalogVersion = catalogObject.getCatalog_version();
-      } else if (!isTopLevelCatalogObject(catalogObject)) {
-        try {
-          addCatalogObject(catalogObject);
-        } catch (Exception e) {
-          LOG.error("Error adding catalog object: " + e.getMessage(), e);
+      // For statestore updates, the service ID and updated version is wrapped in a
+      // CATALOG catalog object.
+      if (obj.type == TCatalogObjectType.CATALOG) {
+        setCatalogServiceId(obj.catalog.catalog_service_id);
+        newCatalogVersion = obj.catalog_version;
+      } else if (!update.first) {
+        // Update top-level objects first.
+        if (isTopLevelCatalogObject(obj)) {
+          updatedObjects.addFirst(obj);
+        } else {
+          updatedObjects.addLast(obj);
+        }
+      } else {
+        // Remove low-level objects first.
+        if (isTopLevelCatalogObject(obj)) {
+          deletedObjects.addLast(obj);
+        } else {
+          deletedObjects.addFirst(obj);
         }
       }
     }
 
-    // Now remove all objects from the catalog. First remove low-level objects (tables,
-    // functions and privileges) and then the top-level objects (databases and roles).
-    for (TCatalogObject catalogObject: req.getRemoved_objects()) {
-      if (!isTopLevelCatalogObject(catalogObject)) {
-        removeCatalogObject(catalogObject);
-      }
-    }
-    for (TCatalogObject catalogObject: req.getRemoved_objects()) {
-      if (isTopLevelCatalogObject(catalogObject)) {
-        removeCatalogObject(catalogObject);
+    for (TCatalogObject catalogObject: updatedObjects) {
+      try {
+        addCatalogObject(catalogObject);
+      } catch (Exception e) {
+        LOG.error("Error adding catalog object: " + e.getMessage(), e);
       }
     }
 
+    for (TCatalogObject catalogObject: deletedObjects) removeCatalogObject(catalogObject);
 
     lastSyncedCatalogVersion_ = newCatalogVersion;
     // Cleanup old entries in the log.
@@ -195,11 +214,11 @@ public class ImpaladCatalog extends Catalog {
     synchronized (catalogUpdateEventNotifier_) {
       catalogUpdateEventNotifier_.notifyAll();
     }
-
     return new TUpdateCatalogCacheResponse(catalogServiceId_,
-        CatalogObjectVersionQueue.INSTANCE.getMinimumVersion());
+        CatalogObjectVersionQueue.INSTANCE.getMinimumVersion(), newCatalogVersion);
   }
 
+
   /**
    * Causes the calling thread to wait until a catalog update notification has been sent
    * or the given timeout has been reached. A timeout value of 0 indicates an indefinite
@@ -248,7 +267,7 @@ public class ImpaladCatalog extends Catalog {
    * This method handles both of these cases.
    */
   public Path getTablePath(org.apache.hadoop.hive.metastore.api.Table msTbl)
-      throws NoSuchObjectException, MetaException, TException {
+      throws TException {
     try (MetaStoreClient msClient = getMetaStoreClient()) {
       // If the table did not have its path set, build the path based on the the
       // location property of the parent database.
@@ -271,7 +290,7 @@ public class ImpaladCatalog extends Catalog {
    *     > than the given TCatalogObject's version.
    */
   private void addCatalogObject(TCatalogObject catalogObject)
-      throws TableLoadingException, DatabaseNotFoundException {
+      throws TableLoadingException {
     // This item is out of date and should not be applied to the catalog.
     if (catalogDeltaLog_.wasObjectRemovedAfter(catalogObject)) {
       if (LOG.isTraceEnabled()) {
@@ -404,6 +423,7 @@ public class ImpaladCatalog extends Catalog {
   }
 
   private void addFunction(TFunction fn, long catalogVersion) {
+    LibCacheSetNeedsRefresh(fn.hdfs_location);
     Function function = Function.fromThrift(fn);
     function.setCatalogVersion(catalogVersion);
     Db db = getDb(function.getFunctionName().getDb());
@@ -427,12 +447,17 @@ public class ImpaladCatalog extends Catalog {
   }
 
   private void addDataSource(TDataSource thrift, long catalogVersion) {
+    LibCacheSetNeedsRefresh(thrift.hdfs_location);
     DataSource dataSource = DataSource.fromThrift(thrift);
     dataSource.setCatalogVersion(catalogVersion);
     addDataSource(dataSource);
   }
 
   private void removeDataSource(TDataSource thrift, long dropCatalogVersion) {
+    DataSource src = dataSources_.get(thrift.name);
+    if (src != null && src.getCatalogVersion() < dropCatalogVersion) {
+      LibCacheRemoveEntry(src.getLocation());
+    }
     removeDataSource(thrift.getName());
   }
 
@@ -468,6 +493,7 @@ public class ImpaladCatalog extends Catalog {
     // version of the drop, remove the function.
     Function fn = db.getFunction(thriftFn.getSignature());
     if (fn != null && fn.getCatalogVersion() < dropCatalogVersion) {
+      LibCacheRemoveEntry(fn.getLocation().getLocation());
       db.removeFunction(thriftFn.getSignature());
       CatalogObjectVersionQueue.INSTANCE.removeVersion(
           fn.getCatalogVersion());
@@ -506,4 +532,15 @@ public class ImpaladCatalog extends Catalog {
   public void setIsReady(boolean isReady) { isReady_.set(isReady); }
   public AuthorizationPolicy getAuthPolicy() { return authPolicy_; }
   public String getDefaultKuduMasterHosts() { return defaultKuduMasterHosts_; }
+
+  private void LibCacheSetNeedsRefresh(String hdfsLocation) {
+    if (!FeSupport.NativeLibCacheSetNeedsRefresh(hdfsLocation)) {
+      LOG.error("NativeLibCacheSetNeedsRefresh(" + hdfsLocation + ") failed.");
+    }
+  }
+  private void LibCacheRemoveEntry(String hdfsLibFile) {
+    if (!FeSupport.NativeLibCacheRemoveEntry(hdfsLibFile)) {
+      LOG.error("LibCacheRemoveEntry(" + hdfsLibFile + ") failed.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/service/FeSupport.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/FeSupport.java b/fe/src/main/java/org/apache/impala/service/FeSupport.java
index 48349c2..b471448 100644
--- a/fe/src/main/java/org/apache/impala/service/FeSupport.java
+++ b/fe/src/main/java/org/apache/impala/service/FeSupport.java
@@ -17,23 +17,18 @@
 
 package org.apache.impala.service;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.impala.analysis.BoolLiteral;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.NullLiteral;
 import org.apache.impala.analysis.SlotRef;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.thrift.TCacheJarParams;
 import org.apache.impala.thrift.TCacheJarResult;
 import org.apache.impala.thrift.TCatalogObject;
@@ -51,6 +46,13 @@ import org.apache.impala.thrift.TSymbolLookupParams;
 import org.apache.impala.thrift.TSymbolLookupResult;
 import org.apache.impala.thrift.TTable;
 import org.apache.impala.util.NativeLibUtil;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -79,6 +81,21 @@ public class FeSupport {
   // Returns a serialized TCacheJarResult
   public native static byte[] NativeCacheJar(byte[] thriftCacheJar);
 
+  // Adds a topic item to the backend's pending metadata-topic update.
+  // 'serializationBuffer' is a serialized TCatalogObject.
+  // The return value is true if the operation succeeds and false otherwise.
+  public native static boolean NativeAddPendingTopicItem(long nativeCatalogServerPtr,
+      String key, byte[] serializationBuffer, boolean deleted);
+
+  // Get a catalog object update from the backend. A pair of isDeletion flag and
+  // serialized TCatalogObject is returned.
+  public native static Pair<Boolean, ByteBuffer> NativeGetNextCatalogObjectUpdate(
+      long nativeIteratorPtr);
+
+  // The return value is true if the operation succeeds and false otherwise.
+  public native static boolean NativeLibCacheSetNeedsRefresh(String hdfsLocation);
+  public native static boolean NativeLibCacheRemoveEntry(String hdfsLibFile);
+
   // Does an RPCs to the Catalog Server to prioritize the metadata loading of a
   // one or more catalog objects. To keep our kerberos configuration consolidated,
   // we make make all RPCs in the BE layer instead of calling the Catalog Server

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index d0936d5..318b248 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -131,6 +131,7 @@ import org.apache.impala.util.MembershipSnapshot;
 import org.apache.impala.util.PatternMatcher;
 import org.apache.impala.util.TResultRowBuilder;
 import org.apache.impala.util.TSessionStateUtil;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -217,7 +218,7 @@ public class Frontend {
   public AuthorizationChecker getAuthzChecker() { return authzChecker_.get(); }
 
   public TUpdateCatalogCacheResponse updateCatalogCache(
-      TUpdateCatalogCacheRequest req) throws CatalogException {
+      TUpdateCatalogCacheRequest req) throws CatalogException, TException {
     if (req.is_delta) return impaladCatalog_.get().updateCatalog(req);
 
     // If this is not a delta, this update should replace the current

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/service/JniCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniCatalog.java b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
index ed5a51a..1d822e4 100644
--- a/fe/src/main/java/org/apache/impala/service/JniCatalog.java
+++ b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
@@ -56,6 +56,7 @@ import org.apache.impala.thrift.TUpdateCatalogRequest;
 import org.apache.impala.thrift.TBackendGflags;
 import org.apache.impala.util.GlogAppender;
 import org.apache.impala.util.PatternMatcher;
+import org.apache.sentry.hdfs.ThriftSerializer;
 import org.apache.thrift.TException;
 import org.apache.thrift.TSerializer;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -118,16 +119,13 @@ public class JniCatalog {
 
   public static TUniqueId getServiceId() { return catalogServiceId_; }
 
-  /**
-   * Gets all catalog objects
-   */
-  public byte[] getCatalogDelta(byte[] thriftGetCatalogDeltaReq)
-      throws ImpalaException, TException {
+  public byte[] getCatalogDelta(byte[] thriftGetCatalogDeltaReq) throws
+      ImpalaException, TException {
     TGetCatalogDeltaRequest params = new TGetCatalogDeltaRequest();
     JniUtil.deserializeThrift(protocolFactory_, params, thriftGetCatalogDeltaReq);
-    TGetCatalogDeltaResponse resp = catalog_.getCatalogDelta(params.getFrom_version());
-    TSerializer serializer = new TSerializer(protocolFactory_);
-    return serializer.serialize(resp);
+    return new TSerializer(protocolFactory_).serialize(new TGetCatalogDeltaResponse(
+        catalog_.getCatalogDelta(params.getNative_catalog_server_ptr(),
+        params.getFrom_version())));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/service/JniFrontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index 688bd0e..3d99a4a 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -168,30 +168,11 @@ public class JniFrontend {
   }
 
   // Deserialize and merge each thrift catalog update into a single merged update
-  public byte[] updateCatalogCache(byte[][] thriftCatalogUpdates) throws ImpalaException {
-    TUniqueId defaultCatalogServiceId = new TUniqueId(0L, 0L);
-    TUpdateCatalogCacheRequest mergedUpdateRequest = new TUpdateCatalogCacheRequest(
-        false, defaultCatalogServiceId, new ArrayList<TCatalogObject>(),
-        new ArrayList<TCatalogObject>());
-    for (byte[] catalogUpdate: thriftCatalogUpdates) {
-      TUpdateCatalogCacheRequest incrementalRequest = new TUpdateCatalogCacheRequest();
-      JniUtil.deserializeThrift(protocolFactory_, incrementalRequest, catalogUpdate);
-      mergedUpdateRequest.is_delta |= incrementalRequest.is_delta;
-      if (!incrementalRequest.getCatalog_service_id().equals(defaultCatalogServiceId)) {
-        mergedUpdateRequest.setCatalog_service_id(
-            incrementalRequest.getCatalog_service_id());
-      }
-      mergedUpdateRequest.getUpdated_objects().addAll(
-          incrementalRequest.getUpdated_objects());
-      mergedUpdateRequest.getRemoved_objects().addAll(
-          incrementalRequest.getRemoved_objects());
-    }
-    TSerializer serializer = new TSerializer(protocolFactory_);
-    try {
-      return serializer.serialize(frontend_.updateCatalogCache(mergedUpdateRequest));
-    } catch (TException e) {
-      throw new InternalException(e.getMessage());
-    }
+  public byte[] updateCatalogCache(byte[] req) throws ImpalaException, TException {
+    TUpdateCatalogCacheRequest request = new TUpdateCatalogCacheRequest();
+    JniUtil.deserializeThrift(protocolFactory_, request, req);
+    return new TSerializer(protocolFactory_).serialize(
+        frontend_.updateCatalogCache(request));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/util/TByteBuffer.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/TByteBuffer.java b/fe/src/main/java/org/apache/impala/util/TByteBuffer.java
new file mode 100644
index 0000000..28d05aa
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/util/TByteBuffer.java
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.util;
+
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+
+/**
+ * ByteBuffer-backed implementation of TTransport. This is copied from thrift 0.10.0.
+ * TODO: Upgrade thrift to 0.10.0 or higher and remove this file.
+ */
+public final class TByteBuffer extends TTransport {
+  private final ByteBuffer byteBuffer;
+
+  public TByteBuffer(ByteBuffer byteBuffer) { this.byteBuffer = byteBuffer; }
+
+  @Override
+  public boolean isOpen() { return true; }
+
+  @Override
+  public void open() {}
+
+  @Override
+  public void close() {}
+
+  @Override
+  public int read(byte[] buf, int off, int len) throws TTransportException {
+    final int n = Math.min(byteBuffer.remaining(), len);
+    if (n > 0) {
+      try {
+        byteBuffer.get(buf, off, n);
+      } catch (BufferUnderflowException e) {
+        throw new TTransportException("Unexpected end of input buffer", e);
+      }
+    }
+    return n;
+  }
+
+  @Override
+  public void write(byte[] buf, int off, int len) throws TTransportException {
+    throw new TTransportException("Write is not supported by TByteBuffer");
+  }
+}


[17/21] impala git commit: Revert "IMPALA-6215: Removes race when using LibCache."

Posted by ta...@apache.org.
Revert "IMPALA-6215: Removes race when using LibCache."

This reverts commit 4aafa5e9ba9fe22d2dbc7764a796b3cd04136cc0.

See IMPALA-6488 for an example of a crash that this revert is
trying  to avoid.

Change-Id: I2e0a22d38f15fb3e34f08633ab0fc7c87c92d40f
Reviewed-on: http://gerrit.cloudera.org:8080/9244
Reviewed-by: Alex Behm <al...@cloudera.com>
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-by: Vuk Ercegovac <ve...@cloudera.com>
Tested-by: Thomas Tauber-Marshall <tm...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: d609fe1b44745abc979dc8b7884ecff356ea73cf
Parents: 1ddb156
Author: Vuk Ercegovac <ve...@cloudera.com>
Authored: Wed Feb 7 11:24:50 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/codegen/llvm-codegen.cc               |  5 +-
 be/src/exec/external-data-source-executor.cc |  5 +-
 be/src/exprs/hive-udf-call.cc                | 57 ++++++++-------
 be/src/exprs/hive-udf-call.h                 |  3 +
 be/src/runtime/lib-cache.cc                  | 22 +++---
 be/src/runtime/lib-cache.h                   | 44 ++----------
 be/src/service/fe-support.cc                 | 11 ++-
 tests/query_test/test_udfs.py                | 88 ++---------------------
 8 files changed, 58 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 72293a7..e1a606c 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -327,10 +327,9 @@ Status LlvmCodeGen::LinkModuleFromLocalFs(const string& file) {
 
 Status LlvmCodeGen::LinkModuleFromHdfs(const string& hdfs_location) {
   if (linked_modules_.find(hdfs_location) != linked_modules_.end()) return Status::OK();
-  LibCacheEntryHandle handle;
   string local_path;
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(hdfs_location, LibCache::TYPE_IR,
-      &handle, &local_path));
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(hdfs_location, LibCache::TYPE_IR,
+      &local_path));
   RETURN_IF_ERROR(LinkModuleFromLocalFs(local_path));
   linked_modules_.insert(hdfs_location);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/exec/external-data-source-executor.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/external-data-source-executor.cc b/be/src/exec/external-data-source-executor.cc
index 20fe50e..7c54f39 100644
--- a/be/src/exec/external-data-source-executor.cc
+++ b/be/src/exec/external-data-source-executor.cc
@@ -136,10 +136,9 @@ ExternalDataSourceExecutor::~ExternalDataSourceExecutor() {
 Status ExternalDataSourceExecutor::Init(const string& jar_path,
     const string& class_name, const string& api_version, const string& init_string) {
   DCHECK(!is_initialized_);
-  LibCacheEntryHandle handle;
   string local_jar_path;
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(
-      jar_path, LibCache::TYPE_JAR, &handle, &local_jar_path));
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
+      jar_path, LibCache::TYPE_JAR, &local_jar_path));
 
   JNIEnv* jni_env = getJNIEnv();
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/exprs/hive-udf-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.cc b/be/src/exprs/hive-udf-call.cc
index e1ac676..19e2e63 100644
--- a/be/src/exprs/hive-udf-call.cc
+++ b/be/src/exprs/hive-udf-call.cc
@@ -174,6 +174,10 @@ Status HiveUdfCall::Init(const RowDescriptor& row_desc, RuntimeState* state) {
   // Initialize children first.
   RETURN_IF_ERROR(ScalarExpr::Init(row_desc, state));
 
+  // Copy the Hive Jar from hdfs to local file system.
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
+      fn_.hdfs_location, LibCache::TYPE_JAR, &local_location_));
+
   // Initialize input_byte_offsets_ and input_buffer_size_
   for (int i = 0; i < GetNumChildren(); ++i) {
     input_byte_offsets_.push_back(input_buffer_size_);
@@ -198,35 +202,30 @@ Status HiveUdfCall::OpenEvaluator(FunctionContext::FunctionStateScope scope,
   JNIEnv* env = getJNIEnv();
   if (env == NULL) return Status("Failed to get/create JVM");
 
-  {
-    LibCacheEntryHandle handle;
-    string local_location;
-    RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(
-        fn_.hdfs_location, LibCache::TYPE_JAR, &handle, &local_location));
-    THiveUdfExecutorCtorParams ctor_params;
-    ctor_params.fn = fn_;
-    ctor_params.local_location = local_location;
-    ctor_params.input_byte_offsets = input_byte_offsets_;
-
-    jni_ctx->input_values_buffer = new uint8_t[input_buffer_size_];
-    jni_ctx->input_nulls_buffer = new uint8_t[GetNumChildren()];
-    jni_ctx->output_value_buffer = new uint8_t[type().GetSlotSize()];
-
-    ctor_params.input_buffer_ptr = (int64_t)jni_ctx->input_values_buffer;
-    ctor_params.input_nulls_ptr = (int64_t)jni_ctx->input_nulls_buffer;
-    ctor_params.output_buffer_ptr = (int64_t)jni_ctx->output_value_buffer;
-    ctor_params.output_null_ptr = (int64_t)&jni_ctx->output_null_value;
-
-    jbyteArray ctor_params_bytes;
-
-    // Add a scoped cleanup jni reference object. This cleans up local refs made below.
-    JniLocalFrame jni_frame;
-    RETURN_IF_ERROR(jni_frame.push(env));
-
-    RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
-    // Create the java executor object
-    jni_ctx->executor = env->NewObject(executor_cl_, executor_ctor_id_, ctor_params_bytes);
-  }
+  THiveUdfExecutorCtorParams ctor_params;
+  ctor_params.fn = fn_;
+  ctor_params.local_location = local_location_;
+  ctor_params.input_byte_offsets = input_byte_offsets_;
+
+  jni_ctx->input_values_buffer = new uint8_t[input_buffer_size_];
+  jni_ctx->input_nulls_buffer = new uint8_t[GetNumChildren()];
+  jni_ctx->output_value_buffer = new uint8_t[type().GetSlotSize()];
+
+  ctor_params.input_buffer_ptr = (int64_t)jni_ctx->input_values_buffer;
+  ctor_params.input_nulls_ptr = (int64_t)jni_ctx->input_nulls_buffer;
+  ctor_params.output_buffer_ptr = (int64_t)jni_ctx->output_value_buffer;
+  ctor_params.output_null_ptr = (int64_t)&jni_ctx->output_null_value;
+
+  jbyteArray ctor_params_bytes;
+
+  // Add a scoped cleanup jni reference object. This cleans up local refs made
+  // below.
+  JniLocalFrame jni_frame;
+  RETURN_IF_ERROR(jni_frame.push(env));
+
+  RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
+  // Create the java executor object
+  jni_ctx->executor = env->NewObject(executor_cl_, executor_ctor_id_, ctor_params_bytes);
   RETURN_ERROR_IF_EXC(env);
   RETURN_IF_ERROR(JniUtil::LocalToGlobalRef(env, jni_ctx->executor, &jni_ctx->executor));
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/exprs/hive-udf-call.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.h b/be/src/exprs/hive-udf-call.h
index 8ca0372..7ce5eb0 100644
--- a/be/src/exprs/hive-udf-call.h
+++ b/be/src/exprs/hive-udf-call.h
@@ -116,6 +116,9 @@ class HiveUdfCall : public ScalarExpr {
   /// error.
   AnyVal* Evaluate(ScalarExprEvaluator* eval, const TupleRow* row) const;
 
+  /// The path on the local FS to the UDF's jar
+  std::string local_location_;
+
   /// input_byte_offsets_[i] is the byte offset child ith's input argument should
   /// be written to.
   std::vector<int> input_byte_offsets_;

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/runtime/lib-cache.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.cc b/be/src/runtime/lib-cache.cc
index b4a4f59..d694c49 100644
--- a/be/src/runtime/lib-cache.cc
+++ b/be/src/runtime/lib-cache.cc
@@ -129,10 +129,6 @@ LibCacheEntry::~LibCacheEntry() {
   unlink(local_path.c_str());
 }
 
-LibCacheEntryHandle::~LibCacheEntryHandle() {
-  if (entry_ != nullptr) LibCache::instance()->DecrementUseCount(entry_);
-}
-
 Status LibCache::GetSoFunctionPtr(const string& hdfs_lib_file, const string& symbol,
     void** fn_ptr, LibCacheEntry** ent, bool quiet) {
   if (hdfs_lib_file.empty()) {
@@ -177,23 +173,21 @@ void LibCache::DecrementUseCount(LibCacheEntry* entry) {
   if (entry == NULL) return;
   bool can_delete = false;
   {
-    unique_lock<mutex> lock(entry->lock);
+    unique_lock<mutex> lock(entry->lock);;
     --entry->use_count;
     can_delete = (entry->use_count == 0 && entry->should_remove);
   }
   if (can_delete) delete entry;
 }
 
-Status LibCache::GetLocalPath(const std::string& hdfs_lib_file, LibType type,
-    LibCacheEntryHandle* handle, string* path) {
-  DCHECK(handle != nullptr && handle->entry() == nullptr);
-  LibCacheEntry* entry = nullptr;
+Status LibCache::GetLocalLibPath(const string& hdfs_lib_file, LibType type,
+                                 string* local_path) {
   unique_lock<mutex> lock;
+  LibCacheEntry* entry = NULL;
   RETURN_IF_ERROR(GetCacheEntry(hdfs_lib_file, type, &lock, &entry));
-  DCHECK(entry != nullptr);
-  ++entry->use_count;
-  handle->SetEntry(entry);
-  *path = entry->local_path;
+  DCHECK(entry != NULL);
+  DCHECK_EQ(entry->type, type);
+  *local_path = entry->local_path;
   return Status::OK();
 }
 
@@ -358,7 +352,7 @@ Status LibCache::GetCacheEntryInternal(const string& hdfs_lib_file, LibType type
     entry_lock->swap(local_entry_lock);
 
     RETURN_IF_ERROR((*entry)->copy_file_status);
-    DCHECK_EQ((*entry)->type, type) << (*entry)->local_path;
+    DCHECK_EQ((*entry)->type, type);
     DCHECK(!(*entry)->local_path.empty());
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/runtime/lib-cache.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.h b/be/src/runtime/lib-cache.h
index 7296a00..4a564ee 100644
--- a/be/src/runtime/lib-cache.h
+++ b/be/src/runtime/lib-cache.h
@@ -49,16 +49,11 @@ class RuntimeState;
 /// using the library. When the caller requests a ptr into the library, they
 /// are given the entry handle and must decrement the ref count when they
 /// are done.
-/// Note: Explicitly managing this reference count at the client is error-prone. See the
-/// api for accessing a path, GetLocalPath(), that uses the handle's scope to manage the
-/// reference count.
 //
 /// TODO:
 /// - refresh libraries
-/// - better cached module management
-/// - improve the api to be less error-prone (IMPALA-6439)
+/// - better cached module management.
 struct LibCacheEntry;
-class LibCacheEntryHandle;
 
 class LibCache {
  public:
@@ -76,16 +71,11 @@ class LibCache {
   /// Initializes the libcache. Must be called before any other APIs.
   static Status Init();
 
-  /// Gets the local 'path' used to cache the file stored at the global 'hdfs_lib_file'.
-  /// If the referenced global file has not been copied locally, it copies it and
-  /// caches the result.
-  ///
-  /// 'handle' must remain in scope while 'path' is used. The reference count to the
-  /// underlying cache entry is decremented when 'handle' goes out-of-scope.
-  ///
-  /// Returns an error if 'hdfs_lib_file' cannot be copied to the local fs.
-  Status GetLocalPath(const std::string& hdfs_lib_file, LibType type,
-      LibCacheEntryHandle* handle, string* path);
+  /// Gets the local file system path for the library at 'hdfs_lib_file'. If
+  /// this file is not already on the local fs, it copies it and caches the
+  /// result. Returns an error if 'hdfs_lib_file' cannot be copied to the local fs.
+  Status GetLocalLibPath(const std::string& hdfs_lib_file, LibType type,
+                         std::string* local_path);
 
   /// Returns status.ok() if the symbol exists in 'hdfs_lib_file', non-ok otherwise.
   /// If 'quiet' is true, the error status for non-Java unfound symbols will not be logged.
@@ -104,7 +94,6 @@ class LibCache {
   /// using fn_ptr and it is no longer valid to use fn_ptr.
   //
   /// If 'quiet' is true, returned error statuses will not be logged.
-  /// TODO: api is error-prone. upgrade to LibCacheEntryHandle (see IMPALA-6439).
   Status GetSoFunctionPtr(const std::string& hdfs_lib_file, const std::string& symbol,
       void** fn_ptr, LibCacheEntry** entry, bool quiet = false);
 
@@ -175,27 +164,6 @@ class LibCache {
                            const LibMap::iterator& entry_iterator);
 };
 
-/// Handle for a LibCacheEntry that decrements its reference count when the handle is
-/// destroyed or re-used for another entry.
-class LibCacheEntryHandle {
- public:
-  LibCacheEntryHandle() {}
-  ~LibCacheEntryHandle();
-
- private:
-  friend class LibCache;
-
-  LibCacheEntry* entry() const { return entry_; }
-  void SetEntry(LibCacheEntry* entry) {
-    if (entry_ != nullptr) LibCache::instance()->DecrementUseCount(entry);
-    entry_ = entry;
-  }
-
-  LibCacheEntry* entry_ = nullptr;
-
-  DISALLOW_COPY_AND_ASSIGN(LibCacheEntryHandle);
-};
-
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index 12ac874..d1979e7 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -296,11 +296,9 @@ static void ResolveSymbolLookup(const TSymbolLookupParams params,
   if (params.fn_binary_type != TFunctionBinaryType::BUILTIN) {
     // Refresh the library if necessary since we're creating a new function
     LibCache::instance()->SetNeedsRefresh(params.location);
-    LibCacheEntryHandle handle;
     string dummy_local_path;
-    Status status = LibCache::instance()->GetLocalPath(
-        params.location, type, &handle, &dummy_local_path);
-
+    Status status = LibCache::instance()->GetLocalLibPath(
+        params.location, type, &dummy_local_path);
     if (!status.ok()) {
       result->__set_result_code(TSymbolLookupResultCode::BINARY_NOT_FOUND);
       result->__set_error_msg(status.GetDetail());
@@ -391,10 +389,9 @@ Java_org_apache_impala_service_FeSupport_NativeCacheJar(
       JniUtil::internal_exc_class(), nullptr);
 
   TCacheJarResult result;
-  LibCacheEntryHandle handle;
   string local_path;
-  Status status = LibCache::instance()->GetLocalPath(
-      params.hdfs_location, LibCache::TYPE_JAR, &handle, &local_path);
+  Status status = LibCache::instance()->GetLocalLibPath(params.hdfs_location,
+      LibCache::TYPE_JAR, &local_path);
   status.ToThrift(&result.status);
   if (status.ok()) result.__set_local_path(local_path);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/d609fe1b/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 61dd54c..1ff716a 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -18,9 +18,6 @@
 from copy import copy
 import os
 import pytest
-import random
-import threading
-import time
 from subprocess import check_call
 
 from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
@@ -319,6 +316,8 @@ class TestUdfExecution(TestUdfBase):
       self.run_test_case('QueryTest/udf-non-deterministic', vector,
           use_db=unique_database)
 
+  # Runs serially as a temporary workaround for IMPALA_6092.
+  @pytest.mark.execute_serially
   def test_java_udfs(self, vector, unique_database):
     self.run_test_case('QueryTest/load-java-udfs', vector, use_db=unique_database)
     self.run_test_case('QueryTest/java-udf', vector, use_db=unique_database)
@@ -419,6 +418,9 @@ class TestUdfTargeted(TestUdfBase):
             unique_database, tgt_udf_path))
     query = "select `{0}`.fn_invalid_symbol('test')".format(unique_database)
 
+    # Dropping the function can interact with other tests whose Java classes are in
+    # the same jar. Use a copy of the jar to avoid unintended interactions.
+    # See IMPALA-6215 and IMPALA-6092 for examples.
     check_call(["hadoop", "fs", "-put", "-f", src_udf_path, tgt_udf_path])
     self.client.execute(drop_fn_stmt)
     self.client.execute(create_fn_stmt)
@@ -427,86 +429,6 @@ class TestUdfTargeted(TestUdfBase):
       assert "Unable to find class" in str(ex)
     self.client.execute(drop_fn_stmt)
 
-  def test_concurrent_jar_drop_use(self, vector, unique_database):
-    """IMPALA-6215: race between dropping/using java udf's defined in the same jar.
-       This test runs concurrent drop/use threads that result in class not found
-       exceptions when the race is present.
-    """
-    udf_src_path = os.path.join(
-      os.environ['IMPALA_HOME'], "testdata/udfs/impala-hive-udfs.jar")
-    udf_tgt_path = get_fs_path(
-      '/test-warehouse/impala-hive-udfs-{0}.jar'.format(unique_database))
-
-    create_fn_to_drop = """create function {0}.foo_{1}() returns string
-                           LOCATION '{2}' SYMBOL='org.apache.impala.TestUpdateUdf'"""
-    create_fn_to_use = """create function {0}.use_it(string) returns string
-                          LOCATION '{1}' SYMBOL='org.apache.impala.TestUdf'"""
-    drop_fn = "drop function if exists {0}.foo_{1}()"
-    use_fn = """select * from (select max(int_col) from functional.alltypesagg
-                where {0}.use_it(string_col) = 'blah' union all
-                (select max(int_col) from functional.alltypesagg
-                 where {0}.use_it(String_col) > '1' union all
-                (select max(int_col) from functional.alltypesagg
-                 where {0}.use_it(string_col) > '1'))) v"""
-    num_drops = 100
-    num_uses = 100
-
-    # use a unique jar for this test to avoid interactions with other tests
-    # that use the same jar
-    check_call(["hadoop", "fs", "-put", "-f", udf_src_path, udf_tgt_path])
-
-    # create all the functions.
-    setup_client = self.create_impala_client()
-    try:
-      s = create_fn_to_use.format(unique_database, udf_tgt_path)
-      print "use create: " + s
-      setup_client.execute(s)
-    except Exception as e:
-      print e
-      assert False
-    for i in range(0, num_drops):
-      try:
-        setup_client.execute(create_fn_to_drop.format(unique_database, i, udf_tgt_path))
-      except Exception as e:
-        print e
-        assert False
-
-    errors = []
-    def use_fn_method():
-      time.sleep(5 + random.random())
-      client = self.create_impala_client()
-      try:
-        client.execute(use_fn.format(unique_database))
-      except Exception as e: errors.append(e)
-
-    def drop_fn_method(i):
-      time.sleep(1 + random.random())
-      client = self.create_impala_client()
-      try:
-        client.execute(drop_fn.format(unique_database, i))
-      except Exception as e: errors.append(e)
-
-    # create threads to use functions.
-    runner_threads = []
-    for i in range(0, num_uses):
-      runner_threads.append(threading.Thread(target=use_fn_method))
-
-    # create threads to drop functions.
-    drop_threads = []
-    for i in range(0, num_drops):
-      runner_threads.append(threading.Thread(target=drop_fn_method, args=(i, )))
-
-    # launch all runner threads.
-    for t in runner_threads: t.start()
-
-    # join all threads.
-    for t in runner_threads: t.join();
-
-    # Check for any errors.
-    for e in errors: print e
-    assert len(errors) == 0
-
-
   @SkipIfLocal.multiple_impalad
   def test_hive_udfs_missing_jar(self, vector, unique_database):
     """ IMPALA-2365: Impalad shouldn't crash if the udf jar isn't present


[05/21] impala git commit: IMPALA-6346: Potential deadlock in KrpcDataStreamMgr

Posted by ta...@apache.org.
IMPALA-6346: Potential deadlock in KrpcDataStreamMgr

In KrpcDataStreamMgr::CreateRecvr() we take the lock_ and
then call recvr->TakeOverEarlySender() for all contexts.
recvr->TakeOverEarlySender() then calls
recvr_->mgr_->EnqueueDeserializeTask((), which can block if the
deserialize pool queue is full. The next thread to become available
in that queue will also have to acquire lock_, thus leading to a
deadlock.

We fix this by moving the EarlySendersList out of the
EarlySendersMap and dropping the lock before taking any actions on
the RPC contexts in the EarlySendersList. All functions called after
dropping 'lock_' do not require the lock to protect them as they are
thread safe.

Additionally modified the BE test data-stream-test to work with KRPC
as well.

Testing: Added a new test to data-stream-test to verify that the
deadlock does not happen. Also, I verified that this test hangs
without the fix.

Change-Id: Ib7d1a8f12a4821092ca61ccc8a6f20c0404d56c7
Reviewed-on: http://gerrit.cloudera.org:8080/8950
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 1a39ab543de25e309d777e64dba8b07e59abac7b
Parents: d181610
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Fri Jan 5 10:48:13 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:52 2018 +0000

----------------------------------------------------------------------
 be/src/runtime/data-stream-test.cc     | 282 +++++++++++++++++++++++-----
 be/src/runtime/krpc-data-stream-mgr.cc |  44 +++--
 2 files changed, 267 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/1a39ab54/be/src/runtime/data-stream-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/data-stream-test.cc b/be/src/runtime/data-stream-test.cc
index 5e70497..07eefd4 100644
--- a/be/src/runtime/data-stream-test.cc
+++ b/be/src/runtime/data-stream-test.cc
@@ -23,13 +23,18 @@
 #include "common/status.h"
 #include "codegen/llvm-codegen.h"
 #include "exprs/slot-ref.h"
+#include "kudu/rpc/rpc_context.h"
+#include "kudu/rpc/service_if.h"
 #include "rpc/auth-provider.h"
 #include "rpc/thrift-server.h"
+#include "rpc/rpc-mgr.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
 #include "runtime/data-stream-mgr-base.h"
 #include "runtime/data-stream-mgr.h"
 #include "runtime/exec-env.h"
+#include "runtime/krpc-data-stream-mgr.h"
+#include "runtime/krpc-data-stream-sender.h"
 #include "runtime/data-stream-sender.h"
 #include "runtime/data-stream-recvr-base.h"
 #include "runtime/data-stream-recvr.h"
@@ -38,6 +43,7 @@
 #include "runtime/backend-client.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/raw-value.inline.h"
+#include "service/data-stream-service.h"
 #include "service/fe-support.h"
 #include "util/cpu-info.h"
 #include "util/disk-info.h"
@@ -47,6 +53,7 @@
 #include "util/mem-info.h"
 #include "util/test-info.h"
 #include "util/tuple-row-compare.h"
+#include "gen-cpp/data_stream_service.pb.h"
 #include "gen-cpp/ImpalaInternalService.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "gen-cpp/Types_types.h"
@@ -54,6 +61,7 @@
 #include "service/fe-support.h"
 
 #include <iostream>
+#include <unistd.h>
 
 #include "common/names.h"
 
@@ -61,9 +69,17 @@ using namespace impala;
 using namespace apache::thrift;
 using namespace apache::thrift::protocol;
 
-DEFINE_int32(port, 20001, "port on which to run Impala test backend");
-DECLARE_string(principal);
+using kudu::MetricEntity;
+using kudu::rpc::ResultTracker;
+using kudu::rpc::RpcContext;
+using kudu::rpc::ServiceIf;
+
+DEFINE_int32(port, 20001, "port on which to run Impala Thrift based test backend.");
 DECLARE_int32(datastream_sender_timeout_ms);
+DECLARE_int32(datastream_service_num_deserialization_threads);
+DECLARE_int32(datastream_service_deserialization_queue_size);
+
+DECLARE_bool(use_krpc);
 
 // We reserve contiguous memory for senders in SetUp. If a test uses more
 // senders, a DCHECK will fail and you should increase this value.
@@ -78,10 +94,12 @@ static const int NUM_BATCHES = TOTAL_DATA_SIZE / BATCH_CAPACITY / PER_ROW_DATA;
 
 namespace impala {
 
-class ImpalaTestBackend : public ImpalaInternalServiceIf {
+// This class acts as a service interface for all Thrift related communication within
+// this test file.
+class ImpalaThriftTestBackend : public ImpalaInternalServiceIf {
  public:
-  ImpalaTestBackend(DataStreamMgr* stream_mgr): mgr_(stream_mgr) {}
-  virtual ~ImpalaTestBackend() {}
+  ImpalaThriftTestBackend(DataStreamMgr* stream_mgr): mgr_(stream_mgr) {}
+  virtual ~ImpalaThriftTestBackend() {}
 
   virtual void ExecQueryFInstances(TExecQueryFInstancesResult& return_val,
       const TExecQueryFInstancesParams& params) {}
@@ -109,13 +127,43 @@ class ImpalaTestBackend : public ImpalaInternalServiceIf {
   DataStreamMgr* mgr_;
 };
 
-class DataStreamTest : public testing::Test {
+// This class acts as a service interface for all KRPC related communication within
+// this test file.
+class ImpalaKRPCTestBackend : public DataStreamServiceIf {
+ public:
+  ImpalaKRPCTestBackend(RpcMgr* rpc_mgr, KrpcDataStreamMgr* stream_mgr)
+    : DataStreamServiceIf(rpc_mgr->metric_entity(), rpc_mgr->result_tracker()),
+      stream_mgr_(stream_mgr) {}
+  virtual ~ImpalaKRPCTestBackend() {}
+
+  virtual void TransmitData(const TransmitDataRequestPB* request,
+      TransmitDataResponsePB* response, RpcContext* rpc_context) {
+    stream_mgr_->AddData(request, response, rpc_context);
+  }
+
+  virtual void EndDataStream(const EndDataStreamRequestPB* request,
+      EndDataStreamResponsePB* response, RpcContext* rpc_context) {
+    stream_mgr_->CloseSender(request, response, rpc_context);
+  }
+
+ private:
+  KrpcDataStreamMgr* stream_mgr_;
+};
+
+template <class T> class DataStreamTestBase : public T {
+ protected:
+  virtual void SetUp() {}
+  virtual void TearDown() {}
+};
+
+enum KrpcSwitch {
+  USE_THRIFT,
+  USE_KRPC
+};
+
+class DataStreamTest : public DataStreamTestBase<testing::TestWithParam<KrpcSwitch> > {
  protected:
   DataStreamTest() : next_val_(0) {
-    // Initialize MemTrackers and RuntimeState for use by the data stream receiver.
-    ABORT_IF_ERROR(exec_env_.InitForFeTests());
-    runtime_state_.reset(new RuntimeState(TQueryCtx(), &exec_env_));
-    mem_pool_.reset(new MemPool(&tracker_));
 
     // Stop tests that rely on mismatched sender / receiver pairs timing out from failing.
     FLAGS_datastream_sender_timeout_ms = 250;
@@ -123,6 +171,14 @@ class DataStreamTest : public testing::Test {
   ~DataStreamTest() { runtime_state_->ReleaseResources(); }
 
   virtual void SetUp() {
+    // Initialize MemTrackers and RuntimeState for use by the data stream receiver.
+    FLAGS_use_krpc = GetParam() == USE_KRPC;
+
+    exec_env_.reset(new ExecEnv());
+    ABORT_IF_ERROR(exec_env_->InitForFeTests());
+    runtime_state_.reset(new RuntimeState(TQueryCtx(), exec_env_.get()));
+    mem_pool_.reset(new MemPool(&tracker_));
+
     CreateRowDesc();
 
     is_asc_.push_back(true);
@@ -131,7 +187,8 @@ class DataStreamTest : public testing::Test {
 
     next_instance_id_.lo = 0;
     next_instance_id_.hi = 0;
-    stream_mgr_ = new DataStreamMgr(new MetricGroup(""));
+    stream_mgr_ = ExecEnv::GetInstance()->stream_mgr();
+    if (GetParam() == USE_KRPC) krpc_mgr_ = ExecEnv::GetInstance()->rpc_mgr();
 
     broadcast_sink_.dest_node_id = DEST_NODE_ID;
     broadcast_sink_.output_partition.type = TPartitionType::UNPARTITIONED;
@@ -159,7 +216,11 @@ class DataStreamTest : public testing::Test {
     // Ensure that individual sender info addresses don't change
     sender_info_.reserve(MAX_SENDERS);
     receiver_info_.reserve(MAX_RECEIVERS);
-    StartBackend();
+    if (GetParam() == USE_THRIFT) {
+      StartThriftBackend();
+    } else {
+      StartKrpcBackend();
+    }
   }
 
   const TDataSink GetSink(TPartitionType::type partition_type) {
@@ -185,8 +246,12 @@ class DataStreamTest : public testing::Test {
     less_than_->Close(runtime_state_.get());
     ScalarExpr::Close(ordering_exprs_);
     mem_pool_->FreeAll();
-    exec_env_.impalad_client_cache()->TestShutdown();
-    StopBackend();
+    if (GetParam() == USE_THRIFT) {
+      exec_env_->impalad_client_cache()->TestShutdown();
+      StopThriftBackend();
+    } else {
+      StopKrpcBackend();
+    }
   }
 
   void Reset() {
@@ -203,7 +268,7 @@ class DataStreamTest : public testing::Test {
   vector<bool> is_asc_;
   vector<bool> nulls_first_;
   TupleRowComparator* less_than_;
-  ExecEnv exec_env_;
+  boost::scoped_ptr<ExecEnv> exec_env_;
   scoped_ptr<RuntimeState> runtime_state_;
   TUniqueId next_instance_id_;
   string stmt_;
@@ -215,8 +280,12 @@ class DataStreamTest : public testing::Test {
   int next_val_;
   int64_t* tuple_mem_;
 
+  // Only used for KRPC. Not owned.
+  RpcMgr* krpc_mgr_ = nullptr;
+  TNetworkAddress krpc_address_;
+
   // receiving node
-  DataStreamMgrBase* stream_mgr_;
+  DataStreamMgrBase* stream_mgr_ = nullptr;
   ThriftServer* server_;
 
   // sending node(s)
@@ -266,6 +335,9 @@ class DataStreamTest : public testing::Test {
     dest.fragment_instance_id = next_instance_id_;
     dest.server.hostname = "127.0.0.1";
     dest.server.port = FLAGS_port;
+    if (GetParam() == USE_KRPC) {
+      dest.__set_krpc_server(krpc_address_);
+    }
     *instance_id = next_instance_id_;
     ++next_instance_id_.lo;
   }
@@ -452,24 +524,51 @@ class DataStreamTest : public testing::Test {
     }
   }
 
-  // Start backend in separate thread.
-  void StartBackend() {
+  // Start Thrift based backend in separate thread.
+  void StartThriftBackend() {
     // Dynamic cast stream_mgr_ which is of type DataStreamMgrBase to derived type
-    // DataStreamMgr, since ImpalaTestBackend() accepts only DataStreamMgr*.
-    boost::shared_ptr<ImpalaTestBackend> handler(
-        new ImpalaTestBackend(dynamic_cast<DataStreamMgr*>(stream_mgr_)));
+    // DataStreamMgr, since ImpalaThriftTestBackend() accepts only DataStreamMgr*.
+    boost::shared_ptr<ImpalaThriftTestBackend> handler(
+        new ImpalaThriftTestBackend(dynamic_cast<DataStreamMgr*>(stream_mgr_)));
     boost::shared_ptr<TProcessor> processor(new ImpalaInternalServiceProcessor(handler));
     ThriftServerBuilder builder("DataStreamTest backend", processor, FLAGS_port);
     ASSERT_OK(builder.Build(&server_));
     ASSERT_OK(server_->Start());
   }
 
-  void StopBackend() {
+  void StartKrpcBackend() {
+    IpAddr ip;
+    ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+    krpc_address_ = MakeNetworkAddress(ip, FLAGS_port);
+
+    MemTracker* data_svc_tracker = obj_pool_.Add(
+        new MemTracker(-1, "Data Stream Service",
+            ExecEnv::GetInstance()->process_mem_tracker()));
+    MemTracker* stream_mgr_tracker = obj_pool_.Add(
+        new MemTracker(-1, "Data Stream Queued RPC Calls",
+            ExecEnv::GetInstance()->process_mem_tracker()));
+
+    KrpcDataStreamMgr* stream_mgr_ref = dynamic_cast<KrpcDataStreamMgr*>(stream_mgr_);
+    ASSERT_OK(stream_mgr_ref->Init(stream_mgr_tracker, data_svc_tracker));
+    ASSERT_OK(krpc_mgr_->Init());
+
+    unique_ptr<ServiceIf> handler(
+        new ImpalaKRPCTestBackend(krpc_mgr_, stream_mgr_ref));
+    ASSERT_OK(krpc_mgr_->RegisterService(CpuInfo::num_cores(), 1024, move(handler),
+        data_svc_tracker));
+    ASSERT_OK(krpc_mgr_->StartServices(krpc_address_));
+  }
+
+  void StopThriftBackend() {
     VLOG_QUERY << "stop backend\n";
     server_->StopForTesting();
     delete server_;
   }
 
+  void StopKrpcBackend() {
+    krpc_mgr_->Shutdown();
+  }
+
   void StartSender(TPartitionType::type partition_type = TPartitionType::UNPARTITIONED,
                    int channel_buffer_size = 1024) {
     VLOG_QUERY << "start sender";
@@ -479,7 +578,7 @@ class DataStreamTest : public testing::Test {
     SenderInfo& info = sender_info_.back();
     info.thread_handle =
         new thread(&DataStreamTest::Sender, this, num_senders, channel_buffer_size,
-                   partition_type);
+                   partition_type, GetParam() == USE_THRIFT);
   }
 
   void JoinSenders() {
@@ -489,35 +588,54 @@ class DataStreamTest : public testing::Test {
     }
   }
 
-  void Sender(
-      int sender_num, int channel_buffer_size, TPartitionType::type partition_type) {
-    RuntimeState state(TQueryCtx(), &exec_env_, desc_tbl_);
+  void Sender(int sender_num,
+      int channel_buffer_size, TPartitionType::type partition_type, bool is_thrift) {
+    RuntimeState state(TQueryCtx(), exec_env_.get(), desc_tbl_);
     VLOG_QUERY << "create sender " << sender_num;
     const TDataSink& sink = GetSink(partition_type);
-    DataStreamSender sender(
-        sender_num, row_desc_, sink.stream_sink, dest_, channel_buffer_size, &state);
+
+    // We create an object of the base class DataSink and cast to the appropriate sender
+    // according to the 'is_thrift' option.
+    scoped_ptr<DataSink> sender;
 
     TExprNode expr_node;
     expr_node.node_type = TExprNodeType::SLOT_REF;
     TExpr output_exprs;
     output_exprs.nodes.push_back(expr_node);
-    EXPECT_OK(sender.Init(vector<TExpr>({output_exprs}), sink, &state));
 
-    EXPECT_OK(sender.Prepare(&state, &tracker_));
-    EXPECT_OK(sender.Open(&state));
+    if (is_thrift) {
+      sender.reset(new DataStreamSender(
+          sender_num, row_desc_, sink.stream_sink, dest_, channel_buffer_size, &state));
+      EXPECT_OK(static_cast<DataStreamSender*>(
+          sender.get())->Init(vector<TExpr>({output_exprs}), sink, &state));
+    } else {
+      sender.reset(new KrpcDataStreamSender(
+          sender_num, row_desc_, sink.stream_sink, dest_, channel_buffer_size, &state));
+      EXPECT_OK(static_cast<KrpcDataStreamSender*>(
+          sender.get())->Init(vector<TExpr>({output_exprs}), sink, &state));
+    }
+
+    EXPECT_OK(sender->Prepare(&state, &tracker_));
+    EXPECT_OK(sender->Open(&state));
     scoped_ptr<RowBatch> batch(CreateRowBatch());
     SenderInfo& info = sender_info_[sender_num];
     int next_val = 0;
     for (int i = 0; i < NUM_BATCHES; ++i) {
       GetNextBatch(batch.get(), &next_val);
       VLOG_QUERY << "sender " << sender_num << ": #rows=" << batch->num_rows();
-      info.status = sender.Send(&state, batch.get());
+      info.status = sender->Send(&state, batch.get());
       if (!info.status.ok()) break;
     }
     VLOG_QUERY << "closing sender" << sender_num;
-    info.status.MergeStatus(sender.FlushFinal(&state));
-    sender.Close(&state);
-    info.num_bytes_sent = sender.GetNumDataBytesSent();
+    info.status.MergeStatus(sender->FlushFinal(&state));
+    sender->Close(&state);
+    if (is_thrift) {
+      info.num_bytes_sent = static_cast<DataStreamSender*>(
+          sender.get())->GetNumDataBytesSent();
+    } else {
+      info.num_bytes_sent = static_cast<KrpcDataStreamSender*>(
+          sender.get())->GetNumDataBytesSent();
+    }
 
     batch->Reset();
     state.ReleaseResources();
@@ -542,7 +660,44 @@ class DataStreamTest : public testing::Test {
   }
 };
 
-TEST_F(DataStreamTest, UnknownSenderSmallResult) {
+// We use a seperate class for tests that are required to be run against Thrift only.
+class DataStreamTestThriftOnly : public DataStreamTest {
+ protected:
+  virtual void SetUp() {
+    DataStreamTest::SetUp();
+  }
+
+  virtual void TearDown() {
+    DataStreamTest::TearDown();
+  }
+};
+
+// We need a seperate test class for IMPALA-6346, since we need to do some pre-SetUp()
+// work. Specifically we need to set 2 flags that will be picked up during the SetUp()
+// phase of the DataStreamTest class.
+class DataStreamTestForImpala6346 : public DataStreamTest {
+ protected:
+  virtual void SetUp() {
+    FLAGS_datastream_service_num_deserialization_threads = 1;
+    FLAGS_datastream_service_deserialization_queue_size = 1;
+    DataStreamTest::SetUp();
+  }
+
+  virtual void TearDown() {
+    DataStreamTest::TearDown();
+  }
+};
+
+INSTANTIATE_TEST_CASE_P(ThriftOrKrpc, DataStreamTest,
+    ::testing::Values(USE_THRIFT, USE_KRPC));
+
+INSTANTIATE_TEST_CASE_P(ThriftOnly, DataStreamTestThriftOnly,
+    ::testing::Values(USE_THRIFT));
+
+INSTANTIATE_TEST_CASE_P(KrpcOnly, DataStreamTestForImpala6346,
+    ::testing::Values(USE_KRPC));
+
+TEST_P(DataStreamTest, UnknownSenderSmallResult) {
   // starting a sender w/o a corresponding receiver results in an error. No bytes should
   // be sent.
   // case 1: entire query result fits in single buffer
@@ -554,7 +709,7 @@ TEST_F(DataStreamTest, UnknownSenderSmallResult) {
   EXPECT_EQ(sender_info_[0].num_bytes_sent, 0);
 }
 
-TEST_F(DataStreamTest, UnknownSenderLargeResult) {
+TEST_P(DataStreamTest, UnknownSenderLargeResult) {
   // case 2: query result requires multiple buffers
   TUniqueId dummy_id;
   GetNextInstanceId(&dummy_id);
@@ -564,7 +719,7 @@ TEST_F(DataStreamTest, UnknownSenderLargeResult) {
   EXPECT_EQ(sender_info_[0].num_bytes_sent, 0);
 }
 
-TEST_F(DataStreamTest, Cancel) {
+TEST_P(DataStreamTest, Cancel) {
   TUniqueId instance_id;
   StartReceiver(TPartitionType::UNPARTITIONED, 1, 1, 1024, false, &instance_id);
   stream_mgr_->Cancel(instance_id);
@@ -575,7 +730,7 @@ TEST_F(DataStreamTest, Cancel) {
   EXPECT_TRUE(receiver_info_[1].status.IsCancelled());
 }
 
-TEST_F(DataStreamTest, BasicTest) {
+TEST_P(DataStreamTest, BasicTest) {
   // TODO: also test that all client connections have been returned
   TPartitionType::type stream_types[] =
       {TPartitionType::UNPARTITIONED, TPartitionType::RANDOM,
@@ -605,8 +760,8 @@ TEST_F(DataStreamTest, BasicTest) {
 // parent is destroyed. In practice the parent is a member of the query's runtime state.
 //
 // TODO: Make lifecycle requirements more explicit.
-TEST_F(DataStreamTest, CloseRecvrWhileReferencesRemain) {
-  scoped_ptr<RuntimeState> runtime_state(new RuntimeState(TQueryCtx(), &exec_env_));
+TEST_P(DataStreamTestThriftOnly, CloseRecvrWhileReferencesRemain) {
+  scoped_ptr<RuntimeState> runtime_state(new RuntimeState(TQueryCtx(), exec_env_.get()));
   RuntimeProfile* profile = RuntimeProfile::Create(&obj_pool_, "TestReceiver");
 
   // Start just one receiver.
@@ -628,7 +783,7 @@ TEST_F(DataStreamTest, CloseRecvrWhileReferencesRemain) {
   // RPC does not cause an error (the receiver will still be called, since it is only
   // Close()'d, not deleted from the data stream manager).
   Status rpc_status;
-  ImpalaBackendConnection client(exec_env_.impalad_client_cache(),
+  ImpalaBackendConnection client(exec_env_->impalad_client_cache(),
       MakeNetworkAddress("localhost", FLAGS_port), &rpc_status);
   EXPECT_OK(rpc_status);
   TTransmitDataParams params;
@@ -647,6 +802,49 @@ TEST_F(DataStreamTest, CloseRecvrWhileReferencesRemain) {
   stream_recvr.reset();
 }
 
+// This test is to exercise a previously present deadlock path which is now fixed, to
+// ensure that the deadlock does not happen anymore. It does this by doing the following:
+// This test starts multiple senders to send to the same receiver. It makes sure that
+// the senders' payloads reach the receiver before the receiver is setup. Once the
+// receiver is being created, it will notice that there are multiple payloads waiting
+// to be processed already and it would hold the KrpcDataStreamMgr::lock_ and call
+// TakeOverEarlySender() which calls EnqueueDeserializeTask() which tries to Offer()
+// the payload to the deserialization_pool_. However, we've set the queue size to 1,
+// which will cause the payload to be stuck on the Offer(). Now any payload that is
+// already being deserialized will be waiting on the KrpcDataStreamMgr::lock_ as well.
+// But the first thread will never release the lock since it's stuck on Offer(), causing
+// a deadlock. This is fixed with IMPALA-6346.
+TEST_P(DataStreamTestForImpala6346, TestNoDeadlock) {
+  TUniqueId instance_id;
+  GetNextInstanceId(&instance_id);
+
+  // Start 4 senders.
+  StartSender(TPartitionType::UNPARTITIONED, 1024 * 1024);
+  StartSender(TPartitionType::UNPARTITIONED, 1024 * 1024);
+  StartSender(TPartitionType::UNPARTITIONED, 1024 * 1024);
+  StartSender(TPartitionType::UNPARTITIONED, 1024 * 1024);
+
+  // Do a small sleep to ensure that the sent payloads reach before the receivers
+  // are created.
+  sleep(2);
+
+  // Setup the receiver.
+  RuntimeProfile* profile = RuntimeProfile::Create(&obj_pool_, "TestReceiver");
+  receiver_info_.push_back(ReceiverInfo(TPartitionType::UNPARTITIONED, 4, 1));
+  ReceiverInfo& info = receiver_info_.back();
+  info.stream_recvr = stream_mgr_->CreateRecvr(runtime_state_.get(), row_desc_,
+      instance_id, DEST_NODE_ID, 4, 1024 * 1024, profile, false);
+  info.thread_handle = new thread(
+      &DataStreamTestForImpala6346_TestNoDeadlock_Test::ReadStream, this, &info);
+
+  JoinSenders();
+  CheckSenders();
+  JoinReceivers();
+
+  // Check that 4 payloads have been received.
+  CheckReceivers(TPartitionType::UNPARTITIONED, 4);
+}
+
 // TODO: more tests:
 // - test case for transmission error in last batch
 // - receivers getting created concurrently

http://git-wip-us.apache.org/repos/asf/impala/blob/1a39ab54/be/src/runtime/krpc-data-stream-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/krpc-data-stream-mgr.cc b/be/src/runtime/krpc-data-stream-mgr.cc
index 3f777ea..fabea13 100644
--- a/be/src/runtime/krpc-data-stream-mgr.cc
+++ b/be/src/runtime/krpc-data-stream-mgr.cc
@@ -54,14 +54,17 @@ DECLARE_int32(datastream_sender_timeout_ms);
 DEFINE_int32(datastream_service_num_deserialization_threads, 16,
     "Number of threads for deserializing RPC requests deferred due to the receiver "
     "not ready or the soft limit of the receiver is reached.");
-
+DEFINE_int32(datastream_service_deserialization_queue_size, 10000,
+    "Number of deferred RPC requests that can be enqueued before being processed by a "
+    "deserialization thread.");
 using boost::mutex;
 
 namespace impala {
 
 KrpcDataStreamMgr::KrpcDataStreamMgr(MetricGroup* metrics)
   : deserialize_pool_("data-stream-mgr", "deserialize",
-      FLAGS_datastream_service_num_deserialization_threads, 10000,
+      FLAGS_datastream_service_num_deserialization_threads,
+      FLAGS_datastream_service_deserialization_queue_size,
       boost::bind(&KrpcDataStreamMgr::DeserializeThreadFn, this, _1, _2)) {
   MetricGroup* dsm_metrics = metrics->GetOrCreateChildGroup("datastream-manager");
   num_senders_waiting_ =
@@ -102,6 +105,7 @@ shared_ptr<DataStreamRecvrBase> KrpcDataStreamMgr::CreateRecvr(
       new KrpcDataStreamRecvr(this, state->instance_mem_tracker(), row_desc,
           finst_id, dest_node_id, num_senders, is_merging, buffer_size, profile));
   uint32_t hash_value = GetHashValue(finst_id, dest_node_id);
+  EarlySendersList early_senders_for_recvr;
   {
     RecvrId recvr_id = make_pair(finst_id, dest_node_id);
     lock_guard<mutex> l(lock_);
@@ -109,25 +113,31 @@ shared_ptr<DataStreamRecvrBase> KrpcDataStreamMgr::CreateRecvr(
     receiver_map_.insert(make_pair(hash_value, recvr));
 
     EarlySendersMap::iterator it = early_senders_map_.find(recvr_id);
+
     if (it != early_senders_map_.end()) {
-      EarlySendersList& early_senders = it->second;
-      // Let the receiver take over the RPC payloads of early senders and process them
-      // asynchronously.
-      for (unique_ptr<TransmitDataCtx>& ctx : early_senders.waiting_sender_ctxs) {
-        // Release memory. The receiver will track it in its instance tracker.
-        int64_t transfer_size = ctx->rpc_context->GetTransferSize();
-        recvr->TakeOverEarlySender(move(ctx));
-        mem_tracker_->Release(transfer_size);
-        num_senders_waiting_->Increment(-1);
-      }
-      for (const unique_ptr<EndDataStreamCtx>& ctx : early_senders.closed_sender_ctxs) {
-        recvr->RemoveSender(ctx->request->sender_id());
-        RespondAndReleaseRpc(Status::OK(), ctx->response, ctx->rpc_context, mem_tracker_);
-        num_senders_waiting_->Increment(-1);
-      }
+      // Move the early senders list here so that we can drop 'lock_'. We need to drop
+      // the lock before processing the early senders to avoid a deadlock.
+      // More details in IMPALA-6346.
+      early_senders_for_recvr = std::move(it->second);
       early_senders_map_.erase(it);
     }
   }
+
+  // Let the receiver take over the RPC payloads of early senders and process them
+  // asynchronously.
+  for (unique_ptr<TransmitDataCtx>& ctx : early_senders_for_recvr.waiting_sender_ctxs) {
+    // Release memory. The receiver will track it in its instance tracker.
+    int64_t transfer_size = ctx->rpc_context->GetTransferSize();
+    recvr->TakeOverEarlySender(move(ctx));
+    mem_tracker_->Release(transfer_size);
+    num_senders_waiting_->Increment(-1);
+  }
+  for (const unique_ptr<EndDataStreamCtx>& ctx :
+      early_senders_for_recvr.closed_sender_ctxs) {
+    recvr->RemoveSender(ctx->request->sender_id());
+    RespondAndReleaseRpc(Status::OK(), ctx->response, ctx->rpc_context, mem_tracker_);
+    num_senders_waiting_->Increment(-1);
+  }
   return recvr;
 }
 


[16/21] impala git commit: Force inlining of BloomFilter::MakeMask

Posted by ta...@apache.org.
Force inlining of BloomFilter::MakeMask

I noticed that this function was showing up in perf top for TPC-H Q8
running locally. It wasn't inlined into BloomFilter::BucketFindAVX2.
Inlining made the query ~5% faster for me locally.

Change-Id: I89282f6c315570bea5ad8a0f854cb6eea0592923
Reviewed-on: http://gerrit.cloudera.org:8080/9214
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 9e887b0aa0c58d79b3a95a8f9097ee09d33a6d19
Parents: 83f9650
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Mon Feb 5 10:57:34 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/benchmarks/bloom-filter-benchmark.cc | 167 ++++++++++++-----------
 be/src/util/bloom-filter.h                  |   3 +-
 2 files changed, 86 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/9e887b0a/be/src/benchmarks/bloom-filter-benchmark.cc
----------------------------------------------------------------------
diff --git a/be/src/benchmarks/bloom-filter-benchmark.cc b/be/src/benchmarks/bloom-filter-benchmark.cc
index c9eef7e..6728c42 100644
--- a/be/src/benchmarks/bloom-filter-benchmark.cc
+++ b/be/src/benchmarks/bloom-filter-benchmark.cc
@@ -42,70 +42,71 @@ using namespace impala;
 // As in bloom-filter.h, ndv refers to the number of unique items inserted into a filter
 // and fpp is the probability of false positives.
 //
-// Machine Info: Intel(R) Core(TM) i7-4790 CPU @ 3.60GHz
+//
+// Machine Info: Intel(R) Core(TM) i7-6700 CPU @ 3.40GHz
 //
 // initialize:                Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           5.92e+03 5.98e+03 6.03e+03         1X         1X         1X
-//            ndv      10k fpp    1.0%           3.17e+03 3.24e+03 3.26e+03     0.535X     0.542X     0.541X
-//            ndv      10k fpp    0.1%           1.16e+03 1.17e+03 1.18e+03     0.195X     0.195X     0.195X
-//            ndv    1000k fpp   10.0%               3.85     3.93     3.93  0.000651X  0.000657X  0.000652X
-//            ndv    1000k fpp    1.0%               2.08     2.12     2.12  0.000351X  0.000354X  0.000351X
-//            ndv    1000k fpp    0.1%               2.08     2.12     2.12  0.000351X  0.000354X  0.000351X
-//            ndv  100000k fpp   10.0%             0.0299   0.0304    0.031  5.06e-06X  5.09e-06X  5.14e-06X
-//            ndv  100000k fpp    1.0%             0.0295   0.0306   0.0311  4.98e-06X  5.12e-06X  5.15e-06X
-//            ndv  100000k fpp    0.1%             0.0151   0.0153   0.0154  2.55e-06X  2.55e-06X  2.55e-06X
+//            ndv      10k fpp   10.0%           5.77e+03 5.81e+03 5.84e+03         1X         1X         1X
+//            ndv      10k fpp    1.0%           3.08e+03  3.1e+03 3.13e+03     0.534X     0.534X     0.536X
+//            ndv      10k fpp    0.1%           1.24e+03 1.25e+03 1.27e+03     0.216X     0.216X     0.217X
+//            ndv    1000k fpp   10.0%               4.71     4.71     4.71  0.000816X  0.000811X  0.000805X
+//            ndv    1000k fpp    1.0%               2.31     2.35     2.35    0.0004X  0.000405X  0.000403X
+//            ndv    1000k fpp    0.1%               2.35     2.35     2.35  0.000408X  0.000405X  0.000403X
+//            ndv  100000k fpp   10.0%             0.0926   0.0935   0.0935  1.61e-05X  1.61e-05X   1.6e-05X
+//            ndv  100000k fpp    1.0%             0.0926   0.0935   0.0935  1.61e-05X  1.61e-05X   1.6e-05X
+//            ndv  100000k fpp    0.1%             0.0481   0.0481   0.0481  8.33e-06X  8.28e-06X  8.23e-06X
 //
 // With AVX2:
 //
 // insert:                    Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           1.22e+05 1.23e+05 1.24e+05         1X         1X         1X
-//            ndv      10k fpp    1.0%           1.22e+05 1.23e+05 1.24e+05     0.998X         1X         1X
-//            ndv      10k fpp    0.1%           1.22e+05 1.23e+05 1.24e+05         1X         1X         1X
-//            ndv    1000k fpp   10.0%           1.16e+05 1.18e+05  1.2e+05      0.95X     0.964X     0.965X
-//            ndv    1000k fpp    1.0%           1.14e+05 1.15e+05 1.16e+05     0.935X     0.941X     0.939X
-//            ndv    1000k fpp    0.1%           1.14e+05 1.16e+05 1.17e+05     0.939X     0.945X     0.943X
-//            ndv  100000k fpp   10.0%           3.35e+04 4.22e+04  5.3e+04     0.275X     0.344X     0.428X
-//            ndv  100000k fpp    1.0%           3.16e+04 4.77e+04 5.78e+04      0.26X     0.388X     0.466X
-//            ndv  100000k fpp    0.1%              3e+04  3.7e+04 4.66e+04     0.246X     0.301X     0.376X
+//            ndv      10k fpp   10.0%            2.1e+05 2.11e+05 2.13e+05         1X         1X         1X
+//            ndv      10k fpp    1.0%           2.16e+05 2.18e+05 2.19e+05      1.03X      1.03X      1.03X
+//            ndv      10k fpp    0.1%           2.12e+05 2.14e+05 2.16e+05      1.01X      1.01X      1.01X
+//            ndv    1000k fpp   10.0%           1.98e+05 1.99e+05 2.01e+05     0.943X     0.942X     0.945X
+//            ndv    1000k fpp    1.0%           1.96e+05 1.98e+05 1.99e+05     0.935X     0.936X     0.937X
+//            ndv    1000k fpp    0.1%           1.96e+05 1.97e+05 1.99e+05     0.935X     0.934X     0.936X
+//            ndv  100000k fpp   10.0%           5.63e+04  5.8e+04 6.18e+04     0.269X     0.274X     0.291X
+//            ndv  100000k fpp    1.0%           5.64e+04 5.84e+04 6.24e+04     0.269X     0.276X     0.293X
+//            ndv  100000k fpp    0.1%           5.56e+04 5.75e+04 5.86e+04     0.265X     0.272X     0.275X
 //
 // find:                      Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//    present ndv      10k fpp   10.0%           1.16e+05 1.17e+05 1.18e+05         1X         1X         1X
-//    absent  ndv      10k fpp   10.0%           1.15e+05 1.17e+05 1.18e+05     0.996X     0.998X         1X
-//    present ndv      10k fpp    1.0%           1.16e+05 1.17e+05 1.18e+05     0.999X     0.996X         1X
-//    absent  ndv      10k fpp    1.0%           1.16e+05 1.17e+05 1.18e+05         1X     0.998X     0.999X
-//    present ndv      10k fpp    0.1%           1.16e+05 1.17e+05 1.18e+05     0.999X     0.997X     0.997X
-//    absent  ndv      10k fpp    0.1%           1.16e+05 1.17e+05 1.18e+05         1X     0.996X     0.998X
-//    present ndv    1000k fpp   10.0%           1.09e+05 1.12e+05 1.14e+05     0.936X     0.958X     0.964X
-//    absent  ndv    1000k fpp   10.0%           1.07e+05 1.14e+05 1.15e+05     0.921X     0.976X     0.976X
-//    present ndv    1000k fpp    1.0%           1.05e+05  1.1e+05 1.12e+05     0.906X     0.943X     0.946X
-//    absent  ndv    1000k fpp    1.0%           1.11e+05 1.13e+05 1.14e+05     0.961X     0.966X     0.969X
-//    present ndv    1000k fpp    0.1%           9.78e+04 1.11e+05 1.12e+05     0.844X     0.944X     0.946X
-//    absent  ndv    1000k fpp    0.1%           1.08e+05 1.13e+05 1.14e+05      0.93X     0.967X      0.97X
-//    present ndv  100000k fpp   10.0%           3.85e+04 4.53e+04 6.12e+04     0.332X     0.387X     0.518X
-//    absent  ndv  100000k fpp   10.0%           2.54e+04 3.01e+04 3.26e+04     0.219X     0.257X     0.276X
-//    present ndv  100000k fpp    1.0%            3.3e+04  4.5e+04 6.06e+04     0.284X     0.384X     0.514X
-//    absent  ndv  100000k fpp    1.0%           2.67e+04 3.01e+04  3.2e+04      0.23X     0.257X     0.271X
-//    present ndv  100000k fpp    0.1%           3.12e+04 4.25e+04 5.15e+04     0.269X     0.362X     0.436X
-//    absent  ndv  100000k fpp    0.1%           2.39e+04 2.69e+04 2.84e+04     0.206X     0.229X      0.24X
+//    present ndv      10k fpp   10.0%           1.97e+05 1.98e+05 1.99e+05         1X         1X         1X
+//    absent  ndv      10k fpp   10.0%           1.99e+05 2.01e+05 2.03e+05      1.01X      1.01X      1.02X
+//    present ndv      10k fpp    1.0%           1.97e+05 1.98e+05    2e+05         1X         1X         1X
+//    absent  ndv      10k fpp    1.0%              2e+05 2.01e+05 2.03e+05      1.02X      1.02X      1.02X
+//    present ndv      10k fpp    0.1%           1.97e+05 1.99e+05    2e+05         1X         1X         1X
+//    absent  ndv      10k fpp    0.1%              2e+05 2.02e+05 2.03e+05      1.02X      1.02X      1.02X
+//    present ndv    1000k fpp   10.0%           1.75e+05 1.77e+05 1.78e+05     0.891X     0.893X     0.893X
+//    absent  ndv    1000k fpp   10.0%           1.78e+05  1.8e+05 1.81e+05     0.907X     0.907X     0.907X
+//    present ndv    1000k fpp    1.0%            1.8e+05 1.82e+05 1.83e+05     0.917X     0.917X     0.919X
+//    absent  ndv    1000k fpp    1.0%           1.84e+05 1.86e+05 1.88e+05     0.937X     0.939X     0.941X
+//    present ndv    1000k fpp    0.1%           1.69e+05  1.7e+05 1.71e+05     0.857X     0.859X     0.858X
+//    absent  ndv    1000k fpp    0.1%            1.7e+05 1.72e+05 1.74e+05     0.866X      0.87X     0.871X
+//    present ndv  100000k fpp   10.0%           5.34e+04 5.53e+04 7.21e+04     0.271X     0.279X     0.362X
+//    absent  ndv  100000k fpp   10.0%           5.05e+04 5.28e+04 5.52e+04     0.257X     0.267X     0.277X
+//    present ndv  100000k fpp    1.0%           5.43e+04 5.74e+04 8.65e+04     0.276X      0.29X     0.434X
+//    absent  ndv  100000k fpp    1.0%           5.09e+04 5.42e+04 5.73e+04     0.259X     0.274X     0.288X
+//    present ndv  100000k fpp    0.1%           5.11e+04 5.24e+04 6.69e+04      0.26X     0.265X     0.336X
+//    absent  ndv  100000k fpp    0.1%           4.93e+04 5.02e+04  5.1e+04     0.251X     0.254X     0.256X
 //
 // union:                     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           5.43e+03 5.63e+03 5.67e+03         1X         1X         1X
-//            ndv      10k fpp    1.0%           2.82e+03 2.84e+03 2.87e+03      0.52X     0.505X     0.507X
-//            ndv      10k fpp    0.1%                780      803      812     0.144X     0.143X     0.143X
-//            ndv    1000k fpp   10.0%               16.2     16.5     16.7   0.00298X   0.00292X   0.00294X
-//            ndv    1000k fpp    1.0%               7.75     8.04     8.11   0.00143X   0.00143X   0.00143X
-//            ndv    1000k fpp    0.1%               7.96     8.11     8.11   0.00147X   0.00144X   0.00143X
-//            ndv  100000k fpp   10.0%              0.045   0.0472   0.0478  8.29e-06X  8.38e-06X  8.44e-06X
-//            ndv  100000k fpp    1.0%              0.045   0.0474   0.0478  8.29e-06X  8.42e-06X  8.44e-06X
-//            ndv  100000k fpp    0.1%              0.023   0.0235   0.0238  4.23e-06X  4.17e-06X   4.2e-06X
+//            ndv      10k fpp   10.0%           6.76e+05  6.8e+05 6.88e+05         1X         1X         1X
+//            ndv      10k fpp    1.0%           6.77e+05 6.81e+05 6.87e+05         1X         1X     0.998X
+//            ndv      10k fpp    0.1%           6.78e+05 6.82e+05 6.86e+05         1X         1X     0.996X
+//            ndv    1000k fpp   10.0%           6.78e+05 6.82e+05 6.88e+05         1X         1X         1X
+//            ndv    1000k fpp    1.0%           6.78e+05 6.83e+05 6.89e+05         1X         1X         1X
+//            ndv    1000k fpp    0.1%           6.77e+05  6.8e+05 6.89e+05         1X         1X         1X
+//            ndv  100000k fpp   10.0%           6.77e+05 6.81e+05 6.88e+05         1X         1X     0.999X
+//            ndv  100000k fpp    1.0%           6.77e+05 6.85e+05 6.89e+05         1X      1.01X         1X
+//            ndv  100000k fpp    0.1%           6.76e+05  6.8e+05 6.88e+05         1X         1X         1X
 //
 //
 // Without AVX or AVX2:
@@ -113,51 +114,51 @@ using namespace impala;
 // insert:                    Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%           9.47e+04 9.52e+04  9.6e+04         1X         1X         1X
-//            ndv      10k fpp    1.0%           9.45e+04 9.53e+04 9.59e+04     0.998X         1X     0.998X
-//            ndv      10k fpp    0.1%            9.2e+04 9.56e+04 9.64e+04     0.972X         1X         1X
-//            ndv    1000k fpp   10.0%            9.2e+04 9.46e+04 9.57e+04     0.972X     0.993X     0.997X
-//            ndv    1000k fpp    1.0%           8.49e+04 9.32e+04 9.45e+04     0.896X     0.979X     0.984X
-//            ndv    1000k fpp    0.1%           8.37e+04 9.35e+04 9.47e+04     0.884X     0.981X     0.986X
-//            ndv  100000k fpp   10.0%           4.03e+04  5.1e+04 5.83e+04     0.425X     0.536X     0.607X
-//            ndv  100000k fpp    1.0%            3.2e+04 3.95e+04 5.11e+04     0.337X     0.415X     0.532X
-//            ndv  100000k fpp    0.1%           3.82e+04 4.52e+04 5.19e+04     0.404X     0.474X      0.54X
+//            ndv      10k fpp   10.0%           9.07e+04 9.12e+04 9.22e+04         1X         1X         1X
+//            ndv      10k fpp    1.0%           9.08e+04 9.13e+04 9.21e+04         1X         1X     0.999X
+//            ndv      10k fpp    0.1%           9.04e+04 9.08e+04 9.15e+04     0.997X     0.996X     0.993X
+//            ndv    1000k fpp   10.0%           8.85e+04 8.92e+04    9e+04     0.976X     0.978X     0.976X
+//            ndv    1000k fpp    1.0%            8.8e+04 8.89e+04 8.94e+04     0.971X     0.975X      0.97X
+//            ndv    1000k fpp    0.1%           8.79e+04 8.83e+04 8.92e+04      0.97X     0.968X     0.968X
+//            ndv  100000k fpp   10.0%           3.64e+04 3.82e+04 4.26e+04     0.401X     0.419X     0.462X
+//            ndv  100000k fpp    1.0%           3.67e+04 3.94e+04 4.52e+04     0.405X     0.432X     0.491X
+//            ndv  100000k fpp    0.1%           3.58e+04 3.75e+04 4.58e+04     0.395X     0.411X     0.497X
 //
 // find:                      Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//    present ndv      10k fpp   10.0%           1.25e+05  1.3e+05 1.31e+05         1X         1X         1X
-//    absent  ndv      10k fpp   10.0%           7.91e+04 7.99e+04 8.06e+04     0.633X     0.614X     0.613X
-//    present ndv      10k fpp    1.0%           1.26e+05 1.32e+05 1.33e+05      1.01X      1.01X      1.01X
-//    absent  ndv      10k fpp    1.0%           9.99e+04 1.01e+05 1.02e+05     0.799X     0.779X     0.777X
-//    present ndv      10k fpp    0.1%           1.25e+05 1.29e+05 1.29e+05     0.999X     0.989X     0.985X
-//    absent  ndv      10k fpp    0.1%           1.52e+05 1.66e+05 1.68e+05      1.21X      1.28X      1.28X
-//    present ndv    1000k fpp   10.0%           9.23e+04 9.61e+04 9.71e+04     0.739X     0.739X     0.739X
-//    absent  ndv    1000k fpp   10.0%           5.77e+04 5.84e+04 5.88e+04     0.462X     0.449X     0.448X
-//    present ndv    1000k fpp    1.0%           7.25e+04 9.08e+04 9.33e+04     0.581X     0.698X      0.71X
-//    absent  ndv    1000k fpp    1.0%            7.6e+04 8.97e+04 9.08e+04     0.608X      0.69X     0.691X
-//    present ndv    1000k fpp    0.1%           8.65e+04 9.35e+04 9.43e+04     0.692X     0.719X     0.717X
-//    absent  ndv    1000k fpp    0.1%           8.33e+04 8.98e+04 9.07e+04     0.667X      0.69X      0.69X
-//    present ndv  100000k fpp   10.0%           2.74e+04 3.06e+04 3.37e+04     0.219X     0.236X     0.256X
-//    absent  ndv  100000k fpp   10.0%           2.88e+04 2.98e+04 3.03e+04     0.231X     0.229X     0.231X
-//    present ndv  100000k fpp    1.0%           2.29e+04 2.82e+04 2.95e+04     0.184X     0.217X     0.224X
-//    absent  ndv  100000k fpp    1.0%           2.84e+04 2.94e+04 3.01e+04     0.227X     0.226X     0.229X
-//    present ndv  100000k fpp    0.1%           2.34e+04 2.72e+04 3.09e+04     0.187X     0.209X     0.235X
-//    absent  ndv  100000k fpp    0.1%            3.3e+04 3.84e+04 3.96e+04     0.264X     0.295X     0.301X
+//    present ndv      10k fpp   10.0%           1.34e+05 1.35e+05 1.36e+05         1X         1X         1X
+//    absent  ndv      10k fpp   10.0%           7.83e+04 7.87e+04 7.94e+04     0.584X     0.583X     0.584X
+//    present ndv      10k fpp    1.0%           1.35e+05 1.36e+05 1.37e+05      1.01X         1X      1.01X
+//    absent  ndv      10k fpp    1.0%           8.79e+04 8.84e+04 8.93e+04     0.656X     0.655X     0.657X
+//    present ndv      10k fpp    0.1%           1.34e+05 1.35e+05 1.36e+05         1X         1X         1X
+//    absent  ndv      10k fpp    0.1%           1.38e+05 1.39e+05  1.4e+05      1.03X      1.03X      1.03X
+//    present ndv    1000k fpp   10.0%            9.6e+04 9.66e+04 9.77e+04     0.716X     0.716X     0.719X
+//    absent  ndv    1000k fpp   10.0%           5.43e+04 5.47e+04 5.51e+04     0.405X     0.405X     0.406X
+//    present ndv    1000k fpp    1.0%           9.48e+04 9.56e+04 9.65e+04     0.707X     0.709X     0.711X
+//    absent  ndv    1000k fpp    1.0%           7.95e+04 8.01e+04 8.06e+04     0.593X     0.593X     0.594X
+//    present ndv    1000k fpp    0.1%           9.47e+04 9.55e+04 9.64e+04     0.707X     0.708X      0.71X
+//    absent  ndv    1000k fpp    0.1%           7.93e+04 7.98e+04 8.05e+04     0.592X     0.592X     0.592X
+//    present ndv  100000k fpp   10.0%           3.34e+04 3.46e+04 3.81e+04     0.249X     0.257X      0.28X
+//    absent  ndv  100000k fpp   10.0%           3.61e+04 3.81e+04 4.04e+04     0.269X     0.282X     0.298X
+//    present ndv  100000k fpp    1.0%           3.86e+04 4.19e+04 4.69e+04     0.288X     0.311X     0.346X
+//    absent  ndv  100000k fpp    1.0%            3.6e+04 3.73e+04 4.12e+04     0.268X     0.276X     0.304X
+//    present ndv  100000k fpp    0.1%           3.59e+04 3.74e+04 3.97e+04     0.268X     0.277X     0.292X
+//    absent  ndv  100000k fpp    0.1%           4.82e+04 4.92e+04 5.11e+04      0.36X     0.365X     0.376X
 //
 // union:                     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                          (relative) (relative) (relative)
 // ---------------------------------------------------------------------------------------------------------
-//            ndv      10k fpp   10.0%            3.9e+03 3.96e+03 3.99e+03         1X         1X         1X
-//            ndv      10k fpp    1.0%            1.9e+03 1.95e+03 1.96e+03     0.487X     0.492X     0.491X
-//            ndv      10k fpp    0.1%                630      638      643     0.161X     0.161X     0.161X
-//            ndv    1000k fpp   10.0%               15.5     15.8     15.9   0.00397X   0.00399X   0.00399X
-//            ndv    1000k fpp    1.0%               7.52     7.74     7.88   0.00193X   0.00196X   0.00197X
-//            ndv    1000k fpp    0.1%               7.46     7.88     7.89   0.00191X   0.00199X   0.00198X
-//            ndv  100000k fpp   10.0%             0.0452   0.0474   0.0478  1.16e-05X   1.2e-05X   1.2e-05X
-//            ndv  100000k fpp    1.0%             0.0452   0.0474   0.0478  1.16e-05X   1.2e-05X   1.2e-05X
-//            ndv  100000k fpp    0.1%             0.0231   0.0235   0.0239  5.92e-06X  5.93e-06X  5.98e-06X
-
+//            ndv      10k fpp   10.0%           6.77e+05 6.81e+05 6.89e+05         1X         1X         1X
+//            ndv      10k fpp    1.0%           6.77e+05 6.82e+05 6.87e+05         1X         1X     0.998X
+//            ndv      10k fpp    0.1%           6.77e+05 6.82e+05 6.89e+05         1X         1X         1X
+//            ndv    1000k fpp   10.0%           6.77e+05  6.8e+05 6.89e+05     0.999X     0.999X         1X
+//            ndv    1000k fpp    1.0%           6.77e+05  6.8e+05 6.88e+05     0.999X     0.999X     0.998X
+//            ndv    1000k fpp    0.1%           6.78e+05 6.82e+05 6.87e+05         1X         1X     0.997X
+//            ndv  100000k fpp   10.0%           6.78e+05 6.82e+05 6.87e+05         1X         1X     0.998X
+//            ndv  100000k fpp    1.0%           6.77e+05  6.8e+05 6.87e+05     0.999X     0.998X     0.998X
+//            ndv  100000k fpp    0.1%           6.77e+05 6.82e+05 6.87e+05     0.999X         1X     0.997X
+//
 // Make a random uint32_t, avoiding the absent high bit and the low-entropy low bits
 // produced by rand().
 uint32_t MakeRand() {

http://git-wip-us.apache.org/repos/asf/impala/blob/9e887b0a/be/src/util/bloom-filter.h
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.h b/be/src/util/bloom-filter.h
index 2e225a2..9628402 100644
--- a/be/src/util/bloom-filter.h
+++ b/be/src/util/bloom-filter.h
@@ -158,7 +158,8 @@ class BloomFilter {
 
   /// A helper function for the AVX2 methods. Turns a 32-bit hash into a 256-bit Bucket
   /// with 1 single 1-bit set in each 32-bit lane.
-  static __m256i MakeMask(const uint32_t hash) __attribute__((__target__("avx2")));
+  static inline ALWAYS_INLINE __m256i MakeMask(const uint32_t hash)
+      __attribute__((__target__("avx2")));
 
   int64_t directory_size() const {
     return 1uLL << (log_num_buckets_ + LOG_BUCKET_BYTE_SIZE);


[20/21] impala git commit: IMPALA-6473: Fix analytic fn that partitions and orders on same expr

Posted by ta...@apache.org.
IMPALA-6473: Fix analytic fn that partitions and orders on same expr

Previously, an analytic function that used the same expr in both the
'partition by' and 'order by' clauses, and where the expr meets the
criteria for being materialized before the sort, would hit an
IllegalStateException due to the expr being inserted into the same
ExprSubstitutionMap twice.

If the values have already been partitioned on the expr, then all of
the values for it in each partition will be the same and also ordering
on the expr doesn't change the results. So, the fix is to simply
exclude the duplicate expr from the 'order by' while still
partitioning on it.

Testing:
- Added a regression test to PlannerTest.

Change-Id: Id5f1d5fbc6f69df5850f96afed345ce27668c30b
Reviewed-on: http://gerrit.cloudera.org:8080/9218
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 2037222cdf9bd730a2ba103eecff7874dc61a702
Parents: e7df3b2
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Mon Feb 5 14:24:23 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 .../apache/impala/planner/AnalyticPlanner.java  | 10 +++++---
 .../queries/PlannerTest/analytic-fns.test       | 27 ++++++++++++++++++--
 .../queries/PlannerTest/insert.test             |  2 +-
 3 files changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/2037222c/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java b/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
index 156bd5b..f504fda 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
@@ -339,9 +339,13 @@ public class AnalyticPlanner {
 
       // then sort on orderByExprs
       for (OrderByElement orderByElement: sortGroup.orderByElements) {
-        sortExprs.add(orderByElement.getExpr());
-        isAsc.add(orderByElement.isAsc());
-        nullsFirst.add(orderByElement.getNullsFirstParam());
+        // If the expr is in the PARTITION BY and already in 'sortExprs', but also in
+        // the ORDER BY, its unnecessary to add it to 'sortExprs' again.
+        if (!sortExprs.contains(orderByElement.getExpr())) {
+          sortExprs.add(orderByElement.getExpr());
+          isAsc.add(orderByElement.isAsc());
+          nullsFirst.add(orderByElement.getNullsFirstParam());
+        }
       }
 
       SortInfo sortInfo = createSortInfo(root, sortExprs, isAsc, nullsFirst);

http://git-wip-us.apache.org/repos/asf/impala/blob/2037222c/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
index 2977f5d..f78bfb2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
@@ -285,7 +285,7 @@ PLAN-ROOT SINK
 |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |
 01:SORT
-|  order by: int_col ASC NULLS FIRST, smallint_col ASC NULLS FIRST, int_col ASC
+|  order by: int_col ASC NULLS FIRST, smallint_col ASC NULLS FIRST
 |
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
@@ -342,7 +342,7 @@ PLAN-ROOT SINK
 |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |
 01:SORT
-|  order by: int_col ASC NULLS FIRST, smallint_col ASC NULLS FIRST, int_col ASC
+|  order by: int_col ASC NULLS FIRST, smallint_col ASC NULLS FIRST
 |
 12:EXCHANGE [HASH(int_col,smallint_col)]
 |
@@ -2434,3 +2434,26 @@ PLAN-ROOT SINK
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ====
+# IMPALA-6473: analytic fn where the same expr is in the 'partition by' and the 'order by'
+select last_value(int_col)
+   over (partition by abs(int_col), string_col order by id, abs(int_col))
+from functional.alltypestiny
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+04:EXCHANGE [UNPARTITIONED]
+|
+02:ANALYTIC
+|  functions: last_value(int_col)
+|  partition by: abs(int_col), string_col
+|  order by: id ASC, abs(int_col) ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|
+01:SORT
+|  order by: abs(int_col) ASC NULLS FIRST, string_col ASC NULLS FIRST, id ASC
+|
+03:EXCHANGE [HASH(abs(int_col),string_col)]
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partitions=4/4 files=4 size=460B
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2037222c/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/insert.test b/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
index 522b058..ac225c2 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/insert.test
@@ -523,7 +523,7 @@ WRITE TO HDFS [functional.alltypestiny, OVERWRITE=false, PARTITION-KEYS=(2009,1)
 |  window: ROWS BETWEEN UNBOUNDED PRECEDING AND 1 PRECEDING
 |
 01:SORT
-|  order by: id ASC NULLS FIRST, id ASC
+|  order by: id ASC NULLS FIRST
 |
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B


[08/21] impala git commit: IMPALA-6471: [docs] Corrected ALTER TABLE ADD PARTITION syntax for Kudu table

Posted by ta...@apache.org.
IMPALA-6471: [docs] Corrected ALTER TABLE ADD PARTITION syntax for Kudu table

Change-Id: I70c49286ed6e250707a6edb5ecd77448d1142d0c
Reviewed-on: http://gerrit.cloudera.org:8080/9187
Reviewed-by: Thomas Tauber-Marshall <tm...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: f258a9125defe9ec6075ce8f469d5b2e9e2eeb1e
Parents: 1a39ab5
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Thu Feb 1 17:15:08 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:52 2018 +0000

----------------------------------------------------------------------
 docs/topics/impala_alter_table.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/f258a912/docs/topics/impala_alter_table.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_alter_table.xml b/docs/topics/impala_alter_table.xml
index 915cdf0..0db9e86 100644
--- a/docs/topics/impala_alter_table.xml
+++ b/docs/topics/impala_alter_table.xml
@@ -76,7 +76,7 @@ ALTER TABLE <varname>name</varname> ALTER [COLUMN] <varname>column_name</varname
 ALTER TABLE <varname>name</varname> ADD [IF NOT EXISTS] PARTITION (<varname>partition_spec</varname>)
   <ph rev="IMPALA-4390">[<varname>location_spec</varname>]</ph>
   <ph rev="IMPALA-4390">[<varname>cache_spec</varname>]</ph>
-<ph rev="kudu">ALTER TABLE <varname>name</varname> ADD [IF NOT EXISTS] RANGE PARTITION (<varname>kudu_partition_spec</varname>)</ph>
+<ph rev="kudu">ALTER TABLE <varname>name</varname> ADD [IF NOT EXISTS] RANGE PARTITION <varname>kudu_partition_spec</varname></ph>
 
 ALTER TABLE <varname>name</varname> DROP [IF EXISTS] PARTITION (<varname>partition_spec</varname>)
   <ph rev="2.3.0">[PURGE]</ph>


[03/21] impala git commit: IMPALA-6476: Skip krpc mem test that uses stress options

Posted by ta...@apache.org.
IMPALA-6476: Skip krpc mem test that uses stress options

Some of the tests added in IMPALA-6193 rely on flags that are only
compiled for debug binaries. This change marks those tests as debug-only
so that they do not break the release tests.

Change-Id: I89ae25ee8c1aca3833c2d98e902ddaad2dd01aad
Reviewed-on: http://gerrit.cloudera.org:8080/9207
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 19ec5757913cf288e3ba2989320f98faa59c4f94
Parents: b59a784
Author: Lars Volker <lv...@cloudera.com>
Authored: Fri Feb 2 17:24:00 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:52 2018 +0000

----------------------------------------------------------------------
 tests/custom_cluster/test_krpc_mem_usage.py | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/19ec5757/tests/custom_cluster/test_krpc_mem_usage.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_krpc_mem_usage.py b/tests/custom_cluster/test_krpc_mem_usage.py
index ed7b056..07efe2f 100644
--- a/tests/custom_cluster/test_krpc_mem_usage.py
+++ b/tests/custom_cluster/test_krpc_mem_usage.py
@@ -71,6 +71,7 @@ class TestKrpcMemUsage(CustomClusterTestSuite):
     # show up in time.
     self.execute_query_verify_mem_usage(self.TEST_QUERY, [DATA_STREAM_SVC_METRIC])
 
+  @SkipIfBuildType.not_dev_build
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args("--use_krpc --stress_datastream_recvr_delay_ms=1000")
   def test_krpc_deferred_memory_usage(self, vector):
@@ -79,6 +80,7 @@ class TestKrpcMemUsage(CustomClusterTestSuite):
     """
     self.execute_query_verify_mem_usage(self.TEST_QUERY, ALL_METRICS)
 
+  @SkipIfBuildType.not_dev_build
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args("--use_krpc --stress_datastream_recvr_delay_ms=1000")
   def test_krpc_deferred_memory_cancellation(self, vector):


[19/21] impala git commit: Revert "IMPALA-6219: Use AES-GCM for spill-to-disk encryption"

Posted by ta...@apache.org.
Revert "IMPALA-6219: Use AES-GCM for spill-to-disk encryption"

This reverts commit 9b68645f9eb9e08899fda860e0946cc05f205479.

Change-Id: Ia06f061a4ecedd1df0d359fe06fe84618b5e07fb
Reviewed-on: http://gerrit.cloudera.org:8080/9226
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 31f38687fa5ab010247e88c2055e00cbeb70712d
Parents: 2037222
Author: Alex Behm <al...@cloudera.com>
Authored: Tue Feb 6 11:50:51 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/runtime/tmp-file-mgr.cc   | 15 ++----
 be/src/util/cpu-info.cc          | 13 +++--
 be/src/util/cpu-info.h           | 13 +++--
 be/src/util/openssl-util-test.cc | 95 +++++++++++-----------------------
 be/src/util/openssl-util.cc      | 96 +++--------------------------------
 be/src/util/openssl-util.h       | 70 ++++++++-----------------
 6 files changed, 74 insertions(+), 228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/runtime/tmp-file-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.cc b/be/src/runtime/tmp-file-mgr.cc
index 3807670..d35d302 100644
--- a/be/src/runtime/tmp-file-mgr.cc
+++ b/be/src/runtime/tmp-file-mgr.cc
@@ -612,26 +612,19 @@ void TmpFileMgr::WriteHandle::WaitForWrite() {
 Status TmpFileMgr::WriteHandle::EncryptAndHash(MemRange buffer) {
   DCHECK(FLAGS_disk_spill_encryption);
   SCOPED_TIMER(encryption_timer_);
-  // Since we're using GCM/CTR/CFB mode, we must take care not to reuse a
+  // Since we're using AES-CTR/AES-CFB mode, we must take care not to reuse a
   // key/IV pair. Regenerate a new key and IV for every data buffer we write.
   key_.InitializeRandom();
   RETURN_IF_ERROR(key_.Encrypt(buffer.data(), buffer.len(), buffer.data()));
-
-  if (!key_.IsGcmMode()) {
-    hash_.Compute(buffer.data(), buffer.len());
-  }
+  hash_.Compute(buffer.data(), buffer.len());
   return Status::OK();
 }
 
 Status TmpFileMgr::WriteHandle::CheckHashAndDecrypt(MemRange buffer) {
   DCHECK(FLAGS_disk_spill_encryption);
   SCOPED_TIMER(encryption_timer_);
-
-  // GCM mode will verify the integrity by itself
-  if (!key_.IsGcmMode()) {
-    if (!hash_.Verify(buffer.data(), buffer.len())) {
-      return Status("Block verification failure");
-    }
+  if (!hash_.Verify(buffer.data(), buffer.len())) {
+    return Status("Block verification failure");
   }
   return key_.Decrypt(buffer.data(), buffer.len(), buffer.data());
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/cpu-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index 1e3fcde..a32571e 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -85,13 +85,12 @@ static struct {
   int64_t flag;
 } flag_mappings[] =
 {
-  { "ssse3",     CpuInfo::SSSE3 },
-  { "sse4_1",    CpuInfo::SSE4_1 },
-  { "sse4_2",    CpuInfo::SSE4_2 },
-  { "popcnt",    CpuInfo::POPCNT },
-  { "avx",       CpuInfo::AVX },
-  { "avx2",      CpuInfo::AVX2 },
-  { "pclmuldqd", CpuInfo::PCLMULQDQ }
+  { "ssse3",  CpuInfo::SSSE3 },
+  { "sse4_1", CpuInfo::SSE4_1 },
+  { "sse4_2", CpuInfo::SSE4_2 },
+  { "popcnt", CpuInfo::POPCNT },
+  { "avx",    CpuInfo::AVX },
+  { "avx2",   CpuInfo::AVX2 },
 };
 static const long num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/cpu-info.h
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.h b/be/src/util/cpu-info.h
index e60babc..38d6782 100644
--- a/be/src/util/cpu-info.h
+++ b/be/src/util/cpu-info.h
@@ -34,13 +34,12 @@ namespace impala {
 /// /sys/devices)
 class CpuInfo {
  public:
-  static const int64_t SSSE3     = (1 << 1);
-  static const int64_t SSE4_1    = (1 << 2);
-  static const int64_t SSE4_2    = (1 << 3);
-  static const int64_t POPCNT    = (1 << 4);
-  static const int64_t AVX       = (1 << 5);
-  static const int64_t AVX2      = (1 << 6);
-  static const int64_t PCLMULQDQ = (1 << 7);
+  static const int64_t SSSE3   = (1 << 1);
+  static const int64_t SSE4_1  = (1 << 2);
+  static const int64_t SSE4_2  = (1 << 3);
+  static const int64_t POPCNT  = (1 << 4);
+  static const int64_t AVX     = (1 << 5);
+  static const int64_t AVX2    = (1 << 6);
 
   /// Cache enums for L1 (data), L2 and L3
   enum CacheLevel {

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/openssl-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util-test.cc b/be/src/util/openssl-util-test.cc
index 76f65a5..8d98b0d 100644
--- a/be/src/util/openssl-util-test.cc
+++ b/be/src/util/openssl-util-test.cc
@@ -44,41 +44,6 @@ class OpenSSLUtilTest : public ::testing::Test {
     }
   }
 
-  /// Fill arbitrary-length buffer with random bytes
-  void GenerateRandomBytes(uint8_t* data, int64_t len) {
-    DCHECK_GE(len, 0);
-    for (int64_t i = 0; i < len; i++) {
-      data[i] = uniform_int_distribution<uint8_t>(0, UINT8_MAX)(rng_);
-    }
-  }
-
-  void TestEncryptionDecryption(const int64_t buffer_size) {
-    vector<uint8_t> original(buffer_size);
-    vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
-    if (buffer_size % 8 == 0) {
-      GenerateRandomData(original.data(), buffer_size);
-    } else {
-      GenerateRandomBytes(original.data(), buffer_size);
-    }
-
-    // Check all the modes
-    AES_CIPHER_MODE modes[] = {AES_256_GCM, AES_256_CTR, AES_256_CFB};
-    for (auto m : modes) {
-      memcpy(scratch.data(), original.data(), buffer_size);
-
-      EncryptionKey key;
-      key.InitializeRandom();
-      key.SetCipherMode(m);
-
-      ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
-      // Check that encryption did something
-      ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
-      ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
-      // Check that we get the original data back.
-      ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
-    }
-  }
-
   mt19937_64 rng_;
 };
 
@@ -92,7 +57,7 @@ TEST_F(OpenSSLUtilTest, Encryption) {
   GenerateRandomData(original.data(), buffer_size);
 
   // Check both CTR & CFB
-  AES_CIPHER_MODE modes[] = {AES_256_GCM, AES_256_CTR, AES_256_CFB};
+  AES_CIPHER_MODE modes[] = {AES_256_CTR, AES_256_CFB};
   for (auto m : modes) {
     // Iterate multiple times to ensure that key regeneration works correctly.
     EncryptionKey key;
@@ -120,42 +85,44 @@ TEST_F(OpenSSLUtilTest, Encryption) {
 /// Test that encryption and decryption work in-place.
 TEST_F(OpenSSLUtilTest, EncryptInPlace) {
   const int buffer_size = 1024 * 1024;
-  TestEncryptionDecryption(buffer_size);
+  vector<uint8_t> original(buffer_size);
+  vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
+
+  EncryptionKey key;
+  // Check both CTR & CFB
+  AES_CIPHER_MODE modes[] = {AES_256_CTR, AES_256_CFB};
+  for (auto m : modes) {
+    GenerateRandomData(original.data(), buffer_size);
+    memcpy(scratch.data(), original.data(), buffer_size);
+
+    key.InitializeRandom();
+    key.SetCipherMode(m);
+
+    ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
+    // Check that encryption did something
+    ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
+    ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
+    // Check that we get the original data back.
+    ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
+  }
 }
 
 /// Test that encryption works with buffer lengths that don't fit in a 32-bit integer.
 TEST_F(OpenSSLUtilTest, EncryptInPlaceHugeBuffer) {
   const int64_t buffer_size = 3 * 1024L * 1024L * 1024L;
-  TestEncryptionDecryption(buffer_size);
-}
-
-/// Test that encryption works with arbitrary-length buffer
-TEST_F(OpenSSLUtilTest, EncryptArbitraryLength) {
-  std::uniform_int_distribution<uint64_t> dis(0, 1024 * 1024);
-  const int buffer_size = dis(rng_);
-  TestEncryptionDecryption(buffer_size);
-}
-
-/// Test integrity in GCM mode
-TEST_F(OpenSSLUtilTest, GcmIntegrity) {
-  const int buffer_size = 1024 * 1024;
-  vector<uint8_t> buffer(buffer_size);
+  vector<uint8_t> original(buffer_size);
+  vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
+  GenerateRandomData(original.data(), buffer_size);
+  memcpy(scratch.data(), original.data(), buffer_size);
 
   EncryptionKey key;
   key.InitializeRandom();
-  key.SetCipherMode(AES_256_GCM);
-
-  // Even it has been set as GCM mode, it may fall back to other modes.
-  // Check if GCM mode is supported at runtime.
-  if (key.IsGcmMode()) {
-    GenerateRandomData(buffer.data(), buffer_size);
-    ASSERT_OK(key.Encrypt(buffer.data(), buffer_size, buffer.data()));
-
-    // tamper the data
-    ++buffer[0];
-    Status s = key.Decrypt(buffer.data(), buffer_size, buffer.data());
-    EXPECT_STR_CONTAINS(s.GetDetail(), "EVP_DecryptFinal");
-  }
+  ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
+  // Check that encryption did something
+  ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
+  ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
+  // Check that we get the original data back.
+  ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
 }
 
 /// Test basic integrity hash functionality.

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/openssl-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.cc b/be/src/util/openssl-util.cc
index ffb47eb..69dc676 100644
--- a/be/src/util/openssl-util.cc
+++ b/be/src/util/openssl-util.cc
@@ -20,7 +20,6 @@
 #include <limits.h>
 #include <sstream>
 
-#include <glog/logging.h>
 #include <openssl/err.h>
 #include <openssl/evp.h>
 #include <openssl/rand.h>
@@ -31,7 +30,6 @@
 #include "gutil/strings/substitute.h"
 
 #include "common/names.h"
-#include "cpu-info.h"
 
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_server_certificate);
@@ -109,20 +107,19 @@ void EncryptionKey::InitializeRandom() {
   }
   RAND_bytes(key_, sizeof(key_));
   RAND_bytes(iv_, sizeof(iv_));
-  memset(gcm_tag_, 0, sizeof(gcm_tag_));
   initialized_ = true;
 }
 
-Status EncryptionKey::Encrypt(const uint8_t* data, int64_t len, uint8_t* out) {
+Status EncryptionKey::Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const {
   return EncryptInternal(true, data, len, out);
 }
 
-Status EncryptionKey::Decrypt(const uint8_t* data, int64_t len, uint8_t* out) {
+Status EncryptionKey::Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const {
   return EncryptInternal(false, data, len, out);
 }
 
 Status EncryptionKey::EncryptInternal(
-    bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) {
+    bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) const {
   DCHECK(initialized_);
   DCHECK_GE(len, 0);
   // Create and initialize the context for encryption
@@ -130,10 +127,6 @@ Status EncryptionKey::EncryptInternal(
   EVP_CIPHER_CTX_init(&ctx);
   EVP_CIPHER_CTX_set_padding(&ctx, 0);
 
-  if (IsGcmMode()) {
-    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_SET_IVLEN, AES_BLOCK_SIZE, NULL);
-  }
-
   // Start encryption/decryption.  We use a 256-bit AES key, and the cipher block mode
   // is either CTR or CFB(stream cipher), both of which support arbitrary length
   // ciphertexts - it doesn't have to be a multiple of 16 bytes. Additionally, CTR
@@ -164,11 +157,6 @@ Status EncryptionKey::EncryptInternal(
     offset += in_len;
   }
 
-  if (IsGcmMode() && !encrypt) {
-    // Set expected tag value
-    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_SET_TAG, AES_BLOCK_SIZE, gcm_tag_);
-  }
-
   // Finalize encryption or decryption.
   int final_out_len;
   success = encrypt ? EVP_EncryptFinal_ex(&ctx, out + offset, &final_out_len) :
@@ -176,93 +164,21 @@ Status EncryptionKey::EncryptInternal(
   if (success != 1) {
     return OpenSSLErr(encrypt ? "EVP_EncryptFinal" : "EVP_DecryptFinal");
   }
-
-  if (IsGcmMode() && encrypt) {
-    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_GET_TAG, AES_BLOCK_SIZE, gcm_tag_);
-  }
-
-  // Again safe due to GCM/CTR/CFB with no padding
+  // Again safe due to CTR/CFB with no padding
   DCHECK_EQ(final_out_len, 0);
   return Status::OK();
 }
 
-/// OpenSSL 1.0.1d
-#define OPENSSL_VERSION_1_0_1D 0x1000104fL
-
-/// If not defined at compile time, define them manually
-/// see: openssl/evp.h
-#ifndef EVP_CIPH_GCM_MODE
-#define EVP_CTRL_GCM_SET_IVLEN 0x9
-#define EVP_CTRL_GCM_GET_TAG 0x10
-#define EVP_CTRL_GCM_SET_TAG 0x11
-#endif
-
 extern "C" {
 ATTRIBUTE_WEAK
 const EVP_CIPHER* EVP_aes_256_ctr();
-
-ATTRIBUTE_WEAK
-const EVP_CIPHER* EVP_aes_256_gcm();
 }
 
 const EVP_CIPHER* EncryptionKey::GetCipher() const {
   // use weak symbol to avoid compiling error on OpenSSL 1.0.0 environment
-  if (mode_ == AES_256_CTR) return EVP_aes_256_ctr();
-  if (mode_ == AES_256_GCM) return EVP_aes_256_gcm();
+  if (mode_ == AES_256_CTR && EVP_aes_256_ctr) return EVP_aes_256_ctr();
 
+  // otherwise, fallback to CFB mode
   return EVP_aes_256_cfb();
 }
-
-void EncryptionKey::SetCipherMode(AES_CIPHER_MODE m) {
-  mode_ = m;
-
-  if (!IsModeSupported(m)) {
-    mode_ = GetSupportedDefaultMode();
-    LOG(WARNING) << Substitute("$0 is not supported, fall back to $1.",
-        ModeToString(m), ModeToString(mode_));
-  }
-}
-
-bool EncryptionKey::IsModeSupported(AES_CIPHER_MODE m) const {
-  switch (m) {
-    case AES_256_GCM:
-      // It becomes a bit tricky for GCM mode, because GCM mode is enabled since
-      // OpenSSL 1.0.1, but the tag validation only works since 1.0.1d. We have
-      // to make sure that OpenSSL version >= 1.0.1d for GCM. So we need
-      // SSLeay(). Note that SSLeay() may return the compiling version on
-      // certain platforms if it was built against an older version(see:
-      // IMPALA-6418). In this case, it will return false, and EncryptionKey
-      // will try to fall back to CTR mode, so it is not ideal but is OK to use
-      // SSLeay() for GCM mode here since in the worst case, we will be using
-      // AES_256_CTR in a system that supports AES_256_GCM.
-      return (CpuInfo::IsSupported(CpuInfo::PCLMULQDQ)
-          && SSLeay() >= OPENSSL_VERSION_1_0_1D && EVP_aes_256_gcm);
-
-    case AES_256_CTR:
-      // If TLS1.2 is supported, then we're on a verison of OpenSSL that
-      // supports AES-256-CTR.
-      return (MaxSupportedTlsVersion() >= TLS1_2_VERSION && EVP_aes_256_ctr);
-
-    case AES_256_CFB:
-      return true;
-
-    default:
-      return false;
-  }
-}
-
-AES_CIPHER_MODE EncryptionKey::GetSupportedDefaultMode() const {
-  if (IsModeSupported(AES_256_GCM)) return AES_256_GCM;
-  if (IsModeSupported(AES_256_CTR)) return AES_256_CTR;
-  return AES_256_CFB;
-}
-
-const string EncryptionKey::ModeToString(AES_CIPHER_MODE m) const {
-  switch(m) {
-    case AES_256_GCM: return "AES-GCM";
-    case AES_256_CTR: return "AES-CTR";
-    case AES_256_CFB: return "AES-CFB";
-  }
-  return "Unknown mode";
-}
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/openssl-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.h b/be/src/util/openssl-util.h
index ef53425..7b1b28e 100644
--- a/be/src/util/openssl-util.h
+++ b/be/src/util/openssl-util.h
@@ -60,9 +60,9 @@ bool IsExternalTlsConfigured();
 void SeedOpenSSLRNG();
 
 enum AES_CIPHER_MODE {
-  AES_256_CFB,
   AES_256_CTR,
-  AES_256_GCM
+  AES_256_CFB,
+  AES_256_GCM // not supported now.
 };
 
 /// The hash of a data buffer used for checking integrity. A SHA256 hash is used
@@ -83,56 +83,43 @@ class IntegrityHash {
 /// The key and initialization vector (IV) required to encrypt and decrypt a buffer of
 /// data. This should be regenerated for each buffer of data.
 ///
-/// We use AES with a 256-bit key and GCM/CTR/CFB cipher block mode, which gives us a
-/// stream cipher that can support arbitrary-length ciphertexts. The mode is chosen
-/// depends on the OpenSSL version & the hardware support at runtime. The IV is used as
+/// We use AES with a 256-bit key and CTR/CFB cipher block mode, which gives us a stream
+/// cipher that can support arbitrary-length ciphertexts. If OpenSSL version at runtime
+/// is 1.0.1 or above, CTR mode is used, otherwise CFB mode is used. The IV is used as
 /// an input to the cipher as the "block to supply before the first block of plaintext".
 /// This is required because all ciphers (except the weak ECB) are built such that each
 /// block depends on the output from the previous block. Since the first block doesn't
 /// have a previous block, we supply this IV. Think of it  as starting off the chain of
 /// encryption.
-///
-/// Notes for GCM:
-/// (1) GCM mode was supported since OpenSSL 1.0.1, however the tag verification
-/// in decryption was only supported since OpenSSL 1.0.1d.
-/// (2) The plaintext and the Additional Authenticated Data(AAD) are the two
-/// categories of data that GCM protects. GCM protects the authenticity of the
-/// plaintext and the AAD, and GCM also protects the confidentiality of the
-/// plaintext. The AAD itself is not required or won't change the security.
-/// In our case(Spill to Disk), we just ignore the AAD.
-
 class EncryptionKey {
  public:
-  EncryptionKey() : initialized_(false) { mode_ = GetSupportedDefaultMode(); }
-
-  /// Initializes a key for temporary use with randomly generated data, and clears the
-  /// tag for GCM mode. Reinitializes with new random values if the key was already
-  /// initialized. We use AES-GCM/AES-CTR/AES-CFB mode so key/IV pairs should not be
-  /// reused. This function automatically reseeds the RNG periodically, so callers do
-  /// not need to do it.
+  EncryptionKey() : initialized_(false) {
+    // If TLS1.2 is supported, then we're on a verison of OpenSSL that supports
+    // AES-256-CTR.
+    mode_ = MaxSupportedTlsVersion() < TLS1_2_VERSION ? AES_256_CFB : AES_256_CTR;
+  }
+
+  /// Initialize a key for temporary use with randomly generated data. Reinitializes with
+  /// new random values if the key was already initialized. We use AES-CTR/AES-CFB mode
+  /// so key/IV pairs should not be reused. This function automatically reseeds the RNG
+  /// periodically, so callers do not need to do it.
   void InitializeRandom();
 
   /// Encrypts a buffer of input data 'data' of length 'len' into an output buffer 'out'.
   /// Exactly 'len' bytes will be written to 'out'. This key must be initialized before
   /// calling. Operates in-place if 'in' == 'out', otherwise the buffers must not overlap.
-  /// For GCM mode, the hash tag will be kept inside(gcm_tag_ variable).
-  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) WARN_UNUSED_RESULT;
+  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Decrypts a buffer of input data 'data' of length 'len' that was encrypted with this
   /// key into an output buffer 'out'. Exactly 'len' bytes will be written to 'out'.
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
-  /// otherwise the buffers must not overlap. For GCM mode, the hash tag, which is
-  /// computed during encryption, will be used for intgerity verification.
-  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) WARN_UNUSED_RESULT;
+  /// otherwise the buffers must not overlap.
+  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Specify a cipher mode. Currently used only for testing but maybe in future we
   /// can provide a configuration option for the end user who can choose a preferred
   /// mode(GCM, CTR, CFB...) based on their software/hardware environment.
-  /// If not supported, fall back to the supported mode at runtime.
-  void SetCipherMode(AES_CIPHER_MODE m);
-
-  /// If is GCM mode at runtime
-  bool IsGcmMode() const { return mode_ == AES_256_GCM; }
+  void SetCipherMode(AES_CIPHER_MODE m) { mode_ = m; }
 
  private:
   /// Helper method that encrypts/decrypts if 'encrypt' is true/false respectively.
@@ -141,25 +128,13 @@ class EncryptionKey {
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
   /// otherwise the buffers must not overlap.
   Status EncryptInternal(bool encrypt, const uint8_t* data, int64_t len,
-      uint8_t* out) WARN_UNUSED_RESULT;
-
-  /// Check if mode m is supported at runtime
-  bool IsModeSupported(AES_CIPHER_MODE m) const;
-
-  /// Returns the a default mode which is supported at runtime. If GCM mode
-  /// is supported, return AES_256_GCM as the default. If GCM is not supported,
-  /// but CTR is still supported, return AES_256_CTR. When both GCM and
-  /// CTR modes are not supported, return AES_256_CFB.
-  AES_CIPHER_MODE GetSupportedDefaultMode() const;
-
-  /// Converts mode type to string.
-  const string ModeToString(AES_CIPHER_MODE m) const;
+      uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Track whether this key has been initialized, to avoid accidentally using
   /// uninitialized keys.
   bool initialized_;
 
-  /// Returns a EVP_CIPHER according to cipher mode at runtime
+  /// return a EVP_CIPHER according to cipher mode at runtime
   const EVP_CIPHER* GetCipher() const;
 
   /// An AES 256-bit key.
@@ -168,9 +143,6 @@ class EncryptionKey {
   /// An initialization vector to feed as the first block to AES.
   uint8_t iv_[AES_BLOCK_SIZE];
 
-  /// Tag for GCM mode
-  uint8_t gcm_tag_[AES_BLOCK_SIZE];
-
   /// Cipher Mode
   AES_CIPHER_MODE mode_;
 };


[09/21] impala git commit: IMPALA-6075: Add Impala daemon metric for catalog version.

Posted by ta...@apache.org.
IMPALA-6075: Add Impala daemon metric for catalog version.

This patch adds new metrics for current version of catalog, current catalog topic version
and catalog service id which are currently used by impala daemon.

Testing:
Verified manually that the new metrics for catalog version, catalog topic version,
catalog service id are displayed and they corresponds to the latest version in
catalogd.INFO log file.

Change-Id: Id2307eb434561ed74ff058106541c0ebda017d97
Reviewed-on: http://gerrit.cloudera.org:8080/8949
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: ffb67b25a599b61a1d02ce2cc9557bcc2ad2da91
Parents: e608ac1
Author: Pranay <ps...@cloudera.com>
Authored: Fri Jan 5 09:31:02 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/service/impala-server.cc |  8 ++++++++
 be/src/service/impala-server.h  |  4 +++-
 be/src/util/impalad-metrics.cc  |  9 +++++++++
 be/src/util/impalad-metrics.h   | 12 ++++++++++++
 common/thrift/metrics.json      | 30 ++++++++++++++++++++++++++++++
 5 files changed, 62 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/ffb67b25/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index cf5f5fb..ee8405b 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -1317,6 +1317,13 @@ Status ImpalaServer::AuthorizeProxyUser(const string& user, const string& do_as_
   return Status(error_msg.str());
 }
 
+void ImpalaServer::CatalogUpdateVersionInfo::UpdateCatalogVersionMetrics()
+{
+  ImpaladMetrics::CATALOG_VERSION->SetValue(catalog_version);
+  ImpaladMetrics::CATALOG_TOPIC_VERSION->SetValue(catalog_topic_version);
+  ImpaladMetrics::CATALOG_SERVICE_ID->SetValue(PrintId(catalog_service_id));
+}
+
 void ImpalaServer::CatalogUpdateCallback(
     const StatestoreSubscriber::TopicDeltaMap& incoming_topic_deltas,
     vector<TTopicDelta>* subscriber_topic_updates) {
@@ -1352,6 +1359,7 @@ void ImpalaServer::CatalogUpdateCallback(
       LOG(INFO) << "Catalog topic update applied with version: " <<
           resp.new_catalog_version << " new min catalog object version: " <<
           resp.min_catalog_object_version;
+      catalog_update_info_.UpdateCatalogVersionMetrics();
     }
     ImpaladMetrics::CATALOG_READY->SetValue(resp.new_catalog_version > 0);
     // TODO: deal with an error status

http://git-wip-us.apache.org/repos/asf/impala/blob/ffb67b25/be/src/service/impala-server.h
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index dad1ebf..237b0cb 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -997,7 +997,9 @@ class ImpalaServer : public ImpalaServiceIf,
       catalog_topic_version(0L),
       min_catalog_object_version(0L) {
     }
-
+    /// Update the metrics to store the current version of catalog, current topic and
+    /// current service id used by impalad.
+    void  UpdateCatalogVersionMetrics();
     /// The last catalog version returned from UpdateCatalog()
     int64_t catalog_version;
     /// The CatalogService ID that this catalog version is from.

http://git-wip-us.apache.org/repos/asf/impala/blob/ffb67b25/be/src/util/impalad-metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/impalad-metrics.cc b/be/src/util/impalad-metrics.cc
index 18e96a8..8f5f1be 100644
--- a/be/src/util/impalad-metrics.cc
+++ b/be/src/util/impalad-metrics.cc
@@ -76,6 +76,9 @@ const char* ImpaladMetricKeys::CATALOG_NUM_DBS =
     "catalog.num-databases";
 const char* ImpaladMetricKeys::CATALOG_NUM_TABLES =
     "catalog.num-tables";
+const char* ImpaladMetricKeys::CATALOG_VERSION = "catalog.curr-version";
+const char* ImpaladMetricKeys::CATALOG_TOPIC_VERSION = "catalog.curr-topic";
+const char* ImpaladMetricKeys::CATALOG_SERVICE_ID = "catalog.curr-serviceid";
 const char* ImpaladMetricKeys::CATALOG_READY =
     "catalog.ready";
 const char* ImpaladMetricKeys::NUM_FILES_OPEN_FOR_INSERT =
@@ -126,6 +129,8 @@ IntCounter* ImpaladMetrics::HEDGED_READ_OPS_WIN = NULL;
 // Gauges
 IntGauge* ImpaladMetrics::CATALOG_NUM_DBS = NULL;
 IntGauge* ImpaladMetrics::CATALOG_NUM_TABLES = NULL;
+IntGauge* ImpaladMetrics::CATALOG_VERSION = NULL;
+IntGauge* ImpaladMetrics::CATALOG_TOPIC_VERSION = NULL;
 IntGauge* ImpaladMetrics::IMPALA_SERVER_NUM_OPEN_BEESWAX_SESSIONS = NULL;
 IntGauge* ImpaladMetrics::IMPALA_SERVER_NUM_OPEN_HS2_SESSIONS = NULL;
 IntGauge* ImpaladMetrics::IO_MGR_NUM_BUFFERS = NULL;
@@ -146,6 +151,7 @@ IntGauge* ImpaladMetrics::RESULTSET_CACHE_TOTAL_BYTES = NULL;
 BooleanProperty* ImpaladMetrics::CATALOG_READY = NULL;
 BooleanProperty* ImpaladMetrics::IMPALA_SERVER_READY = NULL;
 StringProperty* ImpaladMetrics::IMPALA_SERVER_VERSION = NULL;
+StringProperty* ImpaladMetrics::CATALOG_SERVICE_ID = NULL;
 
 // Histograms
 HistogramMetric* ImpaladMetrics::QUERY_DURATIONS = NULL;
@@ -235,6 +241,9 @@ void ImpaladMetrics::CreateMetrics(MetricGroup* m) {
   // Initialize catalog metrics
   CATALOG_NUM_DBS = m->AddGauge(ImpaladMetricKeys::CATALOG_NUM_DBS, 0);
   CATALOG_NUM_TABLES = m->AddGauge(ImpaladMetricKeys::CATALOG_NUM_TABLES, 0);
+  CATALOG_VERSION = m->AddGauge(ImpaladMetricKeys::CATALOG_VERSION, 0);
+  CATALOG_TOPIC_VERSION = m->AddGauge(ImpaladMetricKeys::CATALOG_TOPIC_VERSION, 0);
+  CATALOG_SERVICE_ID = m->AddProperty<string>(ImpaladMetricKeys::CATALOG_SERVICE_ID, "");
   CATALOG_READY = m->AddProperty<bool>(ImpaladMetricKeys::CATALOG_READY, false);
 
   // Maximum duration to be tracked by the query durations metric. No particular reasoning

http://git-wip-us.apache.org/repos/asf/impala/blob/ffb67b25/be/src/util/impalad-metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/impalad-metrics.h b/be/src/util/impalad-metrics.h
index 65ddf1e..f32e3fa 100644
--- a/be/src/util/impalad-metrics.h
+++ b/be/src/util/impalad-metrics.h
@@ -109,6 +109,15 @@ class ImpaladMetricKeys {
   /// Number of DBs in the catalog
   static const char* CATALOG_NUM_DBS;
 
+  /// Current version of catalog with impalad.
+  static const char* CATALOG_VERSION;
+
+  /// Catalog topic version  with impalad.
+  static const char* CATALOG_TOPIC_VERSION;
+
+  /// ServiceID of Catalog with impalad.
+  static const char* CATALOG_SERVICE_ID;
+
   /// Number of tables in the catalog
   static const char* CATALOG_NUM_TABLES;
 
@@ -180,6 +189,8 @@ class ImpaladMetrics {
   // Gauges
   static IntGauge* CATALOG_NUM_DBS;
   static IntGauge* CATALOG_NUM_TABLES;
+  static IntGauge* CATALOG_VERSION;
+  static IntGauge* CATALOG_TOPIC_VERSION;
   static IntGauge* IMPALA_SERVER_NUM_OPEN_BEESWAX_SESSIONS;
   static IntGauge* IMPALA_SERVER_NUM_OPEN_HS2_SESSIONS;
   static IntGauge* IO_MGR_NUM_BUFFERS;
@@ -199,6 +210,7 @@ class ImpaladMetrics {
   static BooleanProperty* CATALOG_READY;
   static BooleanProperty* IMPALA_SERVER_READY;
   static StringProperty* IMPALA_SERVER_VERSION;
+  static StringProperty* CATALOG_SERVICE_ID;
   // Histograms
   static HistogramMetric* QUERY_DURATIONS;
   static HistogramMetric* DDL_DURATIONS;

http://git-wip-us.apache.org/repos/asf/impala/blob/ffb67b25/common/thrift/metrics.json
----------------------------------------------------------------------
diff --git a/common/thrift/metrics.json b/common/thrift/metrics.json
index f493d33..df78a0b 100644
--- a/common/thrift/metrics.json
+++ b/common/thrift/metrics.json
@@ -230,6 +230,36 @@
     "key": "catalog.num-tables"
   },
   {
+    "description": "Catalog topic update version.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Catalog topic update version",
+    "units": "NONE",
+    "kind": "GAUGE",
+    "key": "catalog.curr-version"
+  },
+  {
+    "description": "Statestore topic update version.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Statestore topic update version",
+    "units": "NONE",
+    "kind": "GAUGE",
+    "key": "catalog.curr-topic"
+  },
+  {
+    "description": "Catalog service id.",
+    "contexts": [
+      "IMPALAD"
+    ],
+    "label": "Catalog service id",
+    "units": "NONE",
+    "kind": "PROPERTY",
+    "key": "catalog.curr-serviceid"
+  },
+  {
     "description": "Indicates if the catalog is ready.",
     "contexts": [
       "IMPALAD"


[12/21] impala git commit: IMPALA-6475: Enable running TPCH on Kudu

Posted by ta...@apache.org.
IMPALA-6475: Enable running TPCH on Kudu

Change-Id: I88b66f5db105694b3bcf33360887265996f9059c
Reviewed-on: http://gerrit.cloudera.org:8080/9206
Reviewed-by: Michael Brown <mi...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: d7476703e70b7233513062b8c0c9afaef27d4250
Parents: 19ec575
Author: Taras Bobrovytsky <tb...@cloudera.com>
Authored: Fri Feb 2 16:27:17 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 testdata/workloads/tpch/tpch_core.csv       | 1 +
 testdata/workloads/tpch/tpch_exhaustive.csv | 1 +
 testdata/workloads/tpch/tpch_pairwise.csv   | 1 +
 3 files changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d7476703/testdata/workloads/tpch/tpch_core.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_core.csv b/testdata/workloads/tpch/tpch_core.csv
index 04d7896..86804ac 100644
--- a/testdata/workloads/tpch/tpch_core.csv
+++ b/testdata/workloads/tpch/tpch_core.csv
@@ -7,3 +7,4 @@ file_format:rc, dataset:tpch, compression_codec:none, compression_type:none
 file_format:avro, dataset:tpch, compression_codec: none, compression_type: none
 file_format:avro, dataset:tpch, compression_codec: snap, compression_type: block
 file_format:parquet, dataset:tpch, compression_codec: none, compression_type: none
+file_format:kudu, dataset:tpch, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/d7476703/testdata/workloads/tpch/tpch_exhaustive.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_exhaustive.csv b/testdata/workloads/tpch/tpch_exhaustive.csv
index b9cb980..32085bf 100644
--- a/testdata/workloads/tpch/tpch_exhaustive.csv
+++ b/testdata/workloads/tpch/tpch_exhaustive.csv
@@ -22,3 +22,4 @@ file_format: avro, dataset: tpch, compression_codec: snap, compression_type: blo
 file_format: parquet, dataset: tpch, compression_codec: none, compression_type: none
 file_format: parquet, dataset: tpch, compression_codec: def, compression_type: block
 file_format: parquet, dataset: tpch, compression_codec: snap, compression_type: block
+file_format: kudu, dataset:tpch, compression_codec: none, compression_type: none

http://git-wip-us.apache.org/repos/asf/impala/blob/d7476703/testdata/workloads/tpch/tpch_pairwise.csv
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/tpch_pairwise.csv b/testdata/workloads/tpch/tpch_pairwise.csv
index 46f085c..0744cf5 100644
--- a/testdata/workloads/tpch/tpch_pairwise.csv
+++ b/testdata/workloads/tpch/tpch_pairwise.csv
@@ -13,3 +13,4 @@ file_format: rc, dataset: tpch, compression_codec: def, compression_type: block
 file_format: avro, dataset: tpch, compression_codec: none, compression_type: none
 file_format: parquet, dataset: tpch, compression_codec: none, compression_type: none
 file_format: rc, dataset: tpch, compression_codec: none, compression_type: none
+file_format: kudu, dataset:tpch, compression_codec: none, compression_type: none


[21/21] impala git commit: IMPALA-6477: Fix flakiness with thrift-server-test

Posted by ta...@apache.org.
IMPALA-6477: Fix flakiness with thrift-server-test

THe ThriftKerberizedParamsTest.SslConnectivity backend test verifies
that when kerberos and TLS are switched on, we cannot perform a
negotiation with a non TLS enabled client.

The test works as expected, however, the error messages can be
different in some scenarios. This patch just updates the test to
expect a different error message that has the same effect.

This change also links KRB5_REALM_OVERRIDE back with the
rpc-mgr-test, as one test run showed a similar failure as
IMPALA-6268. It's possible that this is due to IMPALA-6448.

Change-Id: I6294447416ecccc864b842013487f4d93afadc6b
Reviewed-on: http://gerrit.cloudera.org:8080/9247
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: bf892858a088a9490a32ea8692055cbd2ad254fe
Parents: ffb67b2
Author: Sailesh Mukil <sa...@cloudera.com>
Authored: Wed Feb 7 13:26:05 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 21:10:19 2018 +0000

----------------------------------------------------------------------
 be/src/rpc/CMakeLists.txt        |  1 +
 be/src/rpc/thrift-server-test.cc | 10 ++++++++--
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/bf892858/be/src/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/rpc/CMakeLists.txt b/be/src/rpc/CMakeLists.txt
index e4d96e2..1c7adc6 100644
--- a/be/src/rpc/CMakeLists.txt
+++ b/be/src/rpc/CMakeLists.txt
@@ -50,6 +50,7 @@ ADD_BE_TEST(rpc-mgr-test)
 add_dependencies(rpc-mgr-test rpc_test_proto)
 target_link_libraries(rpc-mgr-test rpc_test_proto)
 target_link_libraries(rpc-mgr-test security-test-for-impala)
+target_link_libraries(rpc-mgr-test ${KRB5_REALM_OVERRIDE})
 
 ADD_BE_TEST(rpc-mgr-kerberized-test)
 add_dependencies(rpc-mgr-kerberized-test rpc_test_proto)

http://git-wip-us.apache.org/repos/asf/impala/blob/bf892858/be/src/rpc/thrift-server-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server-test.cc b/be/src/rpc/thrift-server-test.cc
index f1e0f4b..8bd7275 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -172,8 +172,14 @@ TEST_P(ThriftKerberizedParamsTest, SslConnectivity) {
     // When Kerberos is ON, the SASL negotiation happens inside Open(). We expect that to
     // fail beacuse the server expects the client to negotiate over an encrypted
     // connection.
-    EXPECT_STR_CONTAINS(non_ssl_client.Open().GetDetail(),
-        "No more data to read");
+    // The expected error message can either state "No more data to read" or
+    // "Couldn't open transport".
+    const std::string& status = non_ssl_client.Open().GetDetail();
+    size_t found_substr = status.find("No more data to read");
+    if (found_substr == string::npos) {
+      EXPECT_STR_CONTAINS(non_ssl_client.Open().GetDetail(),
+          "Couldn't open transport");
+    }
   }
 
 }


[04/21] impala git commit: IMPALA-6338: Fix flaky test_profile_fragment_instances

Posted by ta...@apache.org.
IMPALA-6338: Fix flaky test_profile_fragment_instances

test_profile_fragment_instances checks that, once all the results have
been returned, every fragment instance appears in the query profile
for a query that internally cancels fragment instances that are still
executing when the results have been fully returned.

Every fis is guaranteed to send a profile to the coordinator in
Finalize(), but previously fragment profiles were not applied by the
coordinator if the backend was 'done', defined as either all instances
having completed or one has entered an error state (including
cancelled).

So, the test could fail by the following sequence:
- Some fragment for a particular backend sends an update to the
  coordinator. 'returned_all_results_' is true, so the coordinator
  responds indicating the the backend should cancel its remaining
  fragments.
- Another fragment from that backend executes Finalize() and reports
  that it was cancelled. This causes the coordinator to consider the
  entire backend to be 'done'.
- A third fragment, which had not previously sent a report from the
  reporting thread, from the same backend executes Finalize(). This
  report will not be applied by the coordinator as the backend is
  considered 'done', so this fragment will not appear in the final
  profile.

The solution is to change the definition of 'done' to not include a
backend that has been cancelled but still has fragments that haven't
completed. This guarantees that for queries that complete successfully
and are cancelled internally, all fis will send a report and have it
applied by the coordinator before all results have been returned,
since if eos is true Coordinator::GetNext() calls
WaitForBackendCompletion(), which in this situation will now wait for
all fis to Finalize().

Returning results for queries that are cancelled by the user is
unaffected as the manual cancel path causes WaitForBackendCompletion().

Testing:
- Ran test_profile_fragment_instances in a loop with no failures.
  I can reliably repro the original problem with a few carefully
  placed sleeps.

Change-Id: I77773a1e3c62952003f37f88fe2b662bb11889ed
Reviewed-on: http://gerrit.cloudera.org:8080/8997
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 4c6a83af3cc4064ce42aa8f74cfd97fb1b9be318
Parents: a243163
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Tue Jan 9 13:59:52 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:52 2018 +0000

----------------------------------------------------------------------
 be/src/common/status.h                      |  6 ++++++
 be/src/runtime/coordinator-backend-state.cc | 12 ++++++------
 be/src/runtime/coordinator-backend-state.h  |  6 ++++--
 be/src/runtime/coordinator.cc               | 10 ++++++----
 be/src/runtime/coordinator.h                |  4 +++-
 tests/query_test/test_observability.py      | 10 +++++-----
 6 files changed, 30 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4c6a83af/be/src/common/status.h
----------------------------------------------------------------------
diff --git a/be/src/common/status.h b/be/src/common/status.h
index 24dba8b..f0f91f7 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -95,7 +95,13 @@ class NODISCARD Status {
   static Status MemLimitExceeded();
   static Status MemLimitExceeded(const std::string& details);
 
+  /// Indicates a 'cancelled' status. CANCELLED should not be reported by a fragment
+  /// instance that encounters a problem - instances should return a specific error,
+  /// and then the coordinator will initiate cancellation.
+  /// TODO: we use this in some places to indicate things other than query cancellation,
+  /// which can be confusing.
   static const Status CANCELLED;
+
   static const Status DEPRECATED_RPC;
 
   /// Copy c'tor makes copy of error detail so Status can be returned by value.

http://git-wip-us.apache.org/repos/asf/impala/blob/4c6a83af/be/src/runtime/coordinator-backend-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index 914a3e4..b238cad 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -234,7 +234,7 @@ void Coordinator::BackendState::LogFirstInProgress(
 }
 
 inline bool Coordinator::BackendState::IsDone() const {
-  return num_remaining_instances_ == 0 || !status_.ok();
+  return num_remaining_instances_ == 0 || (!status_.ok() && !status_.IsCancelled());
 }
 
 bool Coordinator::BackendState::ApplyExecStatusReport(
@@ -338,8 +338,8 @@ bool Coordinator::BackendState::Cancel() {
   // Nothing to cancel if the exec rpc was not sent
   if (!rpc_sent_) return false;
 
-  // don't cancel if it already finished (for any reason)
-  if (IsDone()) return false;
+  // don't cancel if it already finished (for any reason) or cancelled
+  if (IsDone() || status_.IsCancelled()) return false;
 
   /// If the status is not OK, we still try to cancel - !OK status might mean
   /// communication failure between backend and coordinator, but fragment
@@ -391,10 +391,10 @@ bool Coordinator::BackendState::Cancel() {
 void Coordinator::BackendState::PublishFilter(const TPublishFilterParams& rpc_params) {
   DCHECK_EQ(rpc_params.dst_query_id, query_id_);
   {
-    // If the backend is already done, it's not waiting for this filter, so we skip
-    // sending it in this case.
+    // If the backend is already done or cancelled, it's not waiting for this filter, so
+    // we skip sending it in this case.
     lock_guard<mutex> l(lock_);
-    if (IsDone()) return;
+    if (IsDone() || status_.IsCancelled()) return;
   }
 
   if (fragments_.count(rpc_params.dst_fragment_idx) == 0) return;

http://git-wip-us.apache.org/repos/asf/impala/blob/4c6a83af/be/src/runtime/coordinator-backend-state.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator-backend-state.h b/be/src/runtime/coordinator-backend-state.h
index 0973ca3..860b968 100644
--- a/be/src/runtime/coordinator-backend-state.h
+++ b/be/src/runtime/coordinator-backend-state.h
@@ -219,7 +219,7 @@ class Coordinator::BackendState {
 
   /// If the status indicates an error status, execution has either been aborted by the
   /// executing impalad (which then reported the error) or cancellation has been
-  /// initiated; either way, execution must not be cancelled.
+  /// initiated by the coordinator.
   Status status_;
 
   /// Used to distinguish between errors reported by a specific fragment instance,
@@ -254,7 +254,9 @@ class Coordinator::BackendState {
       const FilterRoutingTable& filter_routing_table,
       TExecQueryFInstancesParams* rpc_params);
 
-  /// Return true if execution at this backend is done. Caller must hold lock_.
+  /// Return true if execution at this backend is done. The backend is considered done if
+  /// either all instances have completed, or an error (other than cancel) is encountered.
+  /// Caller must hold lock_.
   bool IsDone() const;
 };
 

http://git-wip-us.apache.org/repos/asf/impala/blob/4c6a83af/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 7973775..05ecf9f 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -867,6 +867,9 @@ Status Coordinator::GetNext(QueryResultSet* results, int max_rows, bool* eos) {
     ReleaseExecResources();
     // wait for all backends to complete before computing the summary
     // TODO: relocate this so GetNext() won't have to wait for backends to complete?
+    // Note that doing this here allows us to ensure that a query that completes
+    // successfully will have a full runtime profile by the time that Fetch() indicates
+    // all of the results have been returned.
     RETURN_IF_ERROR(WaitForBackendCompletion());
     // Release admission control resources after backends are finished.
     ReleaseAdmissionControlResources();
@@ -920,10 +923,8 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
         Substitute("Unknown backend index $0 (max known: $1)",
             params.coord_state_idx, backend_states_.size() - 1));
   }
-  BackendState* backend_state = backend_states_[params.coord_state_idx];
-  // TODO: return here if returned_all_results_?
-  // TODO: return CANCELLED in that case? Although that makes the cancellation propagation
-  // path more irregular.
+  // If the query was cancelled, don't process the update.
+  if (query_status_.IsCancelled()) return Status::OK();
 
   // TODO: only do this when the sink is done; probably missing a done field
   // in TReportExecStatus for that
@@ -931,6 +932,7 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
     UpdateInsertExecStatus(params.insert_exec_status);
   }
 
+  BackendState* backend_state = backend_states_[params.coord_state_idx];
   if (backend_state->ApplyExecStatusReport(params, &exec_summary_, &progress_)) {
     // This report made this backend done, so update the status and
     // num_remaining_backends_.

http://git-wip-us.apache.org/repos/asf/impala/blob/4c6a83af/be/src/runtime/coordinator.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index d630b9a..fbbdfa9 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -290,7 +290,9 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
   boost::mutex lock_;
 
   /// Overall status of the entire query; set to the first reported fragment error
-  /// status or to CANCELLED, if Cancel() is called.
+  /// status or to CANCELLED, if Cancel() is called. Note that some fragments may have
+  /// status CANCELLED even if this is not CANCELLED if cancellation is initiated because
+  /// returned_all_results_ is true or an error is encountered.
   Status query_status_;
 
   /// If true, the query is done returning all results.  It is possible that the

http://git-wip-us.apache.org/repos/asf/impala/blob/4c6a83af/tests/query_test/test_observability.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_observability.py b/tests/query_test/test_observability.py
index 85fc4f1..a7508a4 100644
--- a/tests/query_test/test_observability.py
+++ b/tests/query_test/test_observability.py
@@ -124,14 +124,14 @@ class TestObservability(ImpalaTestSuite):
         join (select * from l LIMIT 2000000) b on a.l_orderkey = -b.l_orderkey;""")
     # There are 3 scan nodes and each appears in the profile 4 times (for 3 fragment
     # instances + the averaged fragment).
-    assert results.runtime_profile.count("HDFS_SCAN_NODE") == 12
+    assert results.runtime_profile.count("HDFS_SCAN_NODE") == 12, results.runtime_profile
     # There are 3 exchange nodes and each appears in the profile 2 times (for 1 fragment
     # instance + the averaged fragment).
-    assert results.runtime_profile.count("EXCHANGE_NODE") == 6
+    assert results.runtime_profile.count("EXCHANGE_NODE") == 6, results.runtime_profile
     # The following appear only in the root fragment which has 1 instance.
-    assert results.runtime_profile.count("HASH_JOIN_NODE") == 2
-    assert results.runtime_profile.count("AGGREGATION_NODE") == 2
-    assert results.runtime_profile.count("PLAN_ROOT_SINK") == 2
+    assert results.runtime_profile.count("HASH_JOIN_NODE") == 2, results.runtime_profile
+    assert results.runtime_profile.count("AGGREGATION_NODE") == 2, results.runtime_profile
+    assert results.runtime_profile.count("PLAN_ROOT_SINK") == 2, results.runtime_profile
 
   # IMPALA-6399: Run this test serially to avoid a delay over the wait time in fetching
   # the profile.


[10/21] impala git commit: IMPALA-6478: Remove garbage NativeAddPendingTopicItem log from catalog

Posted by ta...@apache.org.
IMPALA-6478: Remove garbage NativeAddPendingTopicItem log from catalog

After IMPALA-5990, catalog keeps printing "NativeAddPendingTopicItem
failed" into the log because of the wrongly implemented error handling.
This patch fixes this problem.

Change-Id: Ieb5847a698768c704346371288544a54055bdf1d
Reviewed-on: http://gerrit.cloudera.org:8080/9228
Reviewed-by: Dimitris Tsirogiannis <dt...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: e608ac1b8d4f576c704ad2b915c9312e7ae78cd0
Parents: d609fe1
Author: Tianyi Wang <tw...@cloudera.com>
Authored: Fri Feb 2 18:16:04 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/service/fe-support.cc | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e608ac1b/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index d1979e7..bc4afde 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -425,21 +425,26 @@ Java_org_apache_impala_service_FeSupport_NativeLookupSymbol(
 
 // Add a catalog update to pending_topic_updates_.
 extern "C"
-JNIEXPORT void JNICALL
+JNIEXPORT jboolean JNICALL
 Java_org_apache_impala_service_FeSupport_NativeAddPendingTopicItem(JNIEnv* env,
     jclass caller_class, jlong native_catalog_server_ptr, jstring key,
     jbyteArray serialized_object, jboolean deleted) {
   std::string key_string;
   {
     JniUtfCharGuard key_str;
-    if (!JniUtfCharGuard::create(env, key, &key_str).ok()) return;
+    if (!JniUtfCharGuard::create(env, key, &key_str).ok()) {
+      return static_cast<jboolean>(false);
+    }
     key_string.assign(key_str.get());
   }
   JniScopedArrayCritical obj_buf;
-  if (!JniScopedArrayCritical::Create(env, serialized_object, &obj_buf)) return;
+  if (!JniScopedArrayCritical::Create(env, serialized_object, &obj_buf)) {
+    return static_cast<jboolean>(false);
+  }
   reinterpret_cast<CatalogServer*>(native_catalog_server_ptr)->AddPendingTopicItem(
       std::move(key_string), obj_buf.get(), static_cast<uint32_t>(obj_buf.size()),
       deleted);
+  return static_cast<jboolean>(true);
 }
 
 // Get the next catalog update pointed by 'callback_ctx'.


[02/21] impala git commit: IMPALA-6113: Skip row groups with predicates on NULL columns

Posted by ta...@apache.org.
IMPALA-6113: Skip row groups with predicates on NULL columns

Based on the existing Parquet column chunk level statistics null_count,
Impala's Parquet scanner is enhanced to skip an entire row group if the
null_count statistics indicate that all the values under the predicated
column are NULL as we wouldn't get any result rows from that row group
anyway.

Change-Id: I141317af0e0df30da8f220b29b0bfba364f40ddf
Reviewed-on: http://gerrit.cloudera.org:8080/9140
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: b59a784657c9d6903d5125b686f5c95483d574e4
Parents: 4c6a83a
Author: Gabor Kaszab <ga...@cloudera.com>
Authored: Wed Jan 24 17:01:34 2018 +0100
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:52 2018 +0000

----------------------------------------------------------------------
 be/src/exec/hdfs-parquet-scanner.cc             |  7 +++++
 be/src/exec/parquet-column-stats.cc             | 13 ++++++++
 be/src/exec/parquet-column-stats.h              |  6 ++++
 .../queries/QueryTest/parquet-stats.test        | 33 ++++++++++++++++++++
 4 files changed, 59 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/b59a7846/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index c14edd7..7a10f3c 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -584,6 +584,13 @@ Status HdfsParquetScanner::EvaluateStatsConjuncts(
       DCHECK(false) << "Unsupported function name for statistics evaluation: " << fn_name;
     }
 
+    int64_t null_count = 0;
+    bool null_count_result = ColumnStatsBase::ReadNullCountStat(col_chunk, &null_count);
+    if (null_count_result && null_count == col_chunk.meta_data.num_values) {
+      *skip_row_group = true;
+      break;
+    }
+
     if (stats_read) {
       TupleRow row;
       row.SetTuple(0, min_max_tuple_);

http://git-wip-us.apache.org/repos/asf/impala/blob/b59a7846/be/src/exec/parquet-column-stats.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-column-stats.cc b/be/src/exec/parquet-column-stats.cc
index 4443309..a1d1155 100644
--- a/be/src/exec/parquet-column-stats.cc
+++ b/be/src/exec/parquet-column-stats.cc
@@ -129,6 +129,19 @@ bool ColumnStatsBase::ReadFromThrift(const parquet::ColumnChunk& col_chunk,
   return false;
 }
 
+bool ColumnStatsBase::ReadNullCountStat(const parquet::ColumnChunk& col_chunk,
+    int64_t* null_count) {
+  if (!(col_chunk.__isset.meta_data && col_chunk.meta_data.__isset.statistics)) {
+    return false;
+  }
+  const parquet::Statistics& stats = col_chunk.meta_data.statistics;
+  if (stats.__isset.null_count) {
+    *null_count = stats.null_count;
+    return true;
+  }
+  return false;
+}
+
 Status ColumnStatsBase::CopyToBuffer(StringBuffer* buffer, StringValue* value) {
   if (value->ptr == buffer->buffer()) return Status::OK();
   buffer->Clear();

http://git-wip-us.apache.org/repos/asf/impala/blob/b59a7846/be/src/exec/parquet-column-stats.h
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-column-stats.h b/be/src/exec/parquet-column-stats.h
index 0ff277c..e9cf801 100644
--- a/be/src/exec/parquet-column-stats.h
+++ b/be/src/exec/parquet-column-stats.h
@@ -73,6 +73,12 @@ class ColumnStatsBase {
       const ColumnType& col_type, const parquet::ColumnOrder* col_order,
       StatsField stats_field, void* slot);
 
+  // Gets the null_count statistics from the given column chunk's metadata and returns
+  // it via an output parameter.
+  // Returns true if the null_count stats were read successfully, false otherwise.
+  static bool ReadNullCountStat(const parquet::ColumnChunk& col_chunk,
+      int64_t* null_count);
+
   /// Merges this statistics object with values from 'other'. If other has not been
   /// initialized, then this object will not be changed.
   virtual void Merge(const ColumnStatsBase& other) = 0;

http://git-wip-us.apache.org/repos/asf/impala/blob/b59a7846/testdata/workloads/functional-query/queries/QueryTest/parquet-stats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/parquet-stats.test b/testdata/workloads/functional-query/queries/QueryTest/parquet-stats.test
index d03b4c9..70b5f27 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/parquet-stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/parquet-stats.test
@@ -458,3 +458,36 @@ select count(*) from functional_parquet.alltypes where id < 0;
 aggregation(SUM, NumRowGroups): 24
 aggregation(SUM, NumStatsFilteredRowGroups): 0
 ====
+---- QUERY
+# Check that all the row groups are skipped using null_count stat
+create table table_for_null_count_test (i int, j int) stored as parquet;
+insert into table_for_null_count_test values (1, NULL), (2, NULL), (3, NULL);
+select count(*) from table_for_null_count_test where j < 3;
+---- RESULTS
+0
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+aggregation(SUM, NumStatsFilteredRowGroups): 1
+====
+---- QUERY
+# Insert another row group where not all the 'j' values are NULL
+insert into table_for_null_count_test values (4, 1), (5, NULL);
+select i from table_for_null_count_test where j < 3;
+---- RESULTS
+4
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+aggregation(SUM, NumStatsFilteredRowGroups): 1
+====
+---- QUERY
+# Turning off parquet stats and verifying that no row groups are skipped
+set PARQUET_READ_STATISTICS=0;
+create table table_for_null_count_test2 (i int, j int) stored as parquet;
+insert into table_for_null_count_test2 values (1, NULL), (2, NULL), (3, NULL);
+select count(*) from table_for_null_count_test2 where j < 3;
+---- RESULTS
+0
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+aggregation(SUM, NumStatsFilteredRowGroups): 0
+====


[18/21] impala git commit: IMPALA-6448: Re-enable kerberized testing with KRPC

Posted by ta...@apache.org.
IMPALA-6448: Re-enable kerberized testing with KRPC

For the patch for IMPALA-5054, we realized that we needed to make
the kudu::rpc::Messenger configurable. A patch was done on the Kudu
side which is tracked by KUDU-2228. As part of that patch, one of
the design decisions taken was to only allow kerberos either on or
off for the entirety of the process life. This means that we cannot
switch kerberos on and off in the same process any more with KRPC.
This behavior can be found in SaslInit() in kudu/rpc/sasl_common.cc
as SaslInit() which is called once per process will hard code some
configuration which cannot be toggled.

This affected our kerberized rpc-mgr-tests. This patch splits out
the kerberized part of rpc-mgr-test into rpc-mgr-kerberized-test.

It also puts the common code between both the files into
rpc-mgr-test-base.h

Change-Id: I6412978316de90875c98f8fbe51c8d215c227b18
Reviewed-on: http://gerrit.cloudera.org:8080/9164
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: d84657baa14f166a0b750c9bf410711fd27c6d43
Parents: ee74a62
Author: Sailesh Mukil <sa...@apache.org>
Authored: Tue Jan 30 21:35:28 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/rpc/CMakeLists.txt             |   7 +-
 be/src/rpc/rpc-mgr-kerberized-test.cc |  89 ++++++++
 be/src/rpc/rpc-mgr-test-base.h        | 269 +++++++++++++++++++++++
 be/src/rpc/rpc-mgr-test.cc            | 333 ++---------------------------
 4 files changed, 381 insertions(+), 317 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/d84657ba/be/src/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/rpc/CMakeLists.txt b/be/src/rpc/CMakeLists.txt
index 7beb80d..e4d96e2 100644
--- a/be/src/rpc/CMakeLists.txt
+++ b/be/src/rpc/CMakeLists.txt
@@ -50,7 +50,12 @@ ADD_BE_TEST(rpc-mgr-test)
 add_dependencies(rpc-mgr-test rpc_test_proto)
 target_link_libraries(rpc-mgr-test rpc_test_proto)
 target_link_libraries(rpc-mgr-test security-test-for-impala)
-target_link_libraries(rpc-mgr-test ${KRB5_REALM_OVERRIDE})
+
+ADD_BE_TEST(rpc-mgr-kerberized-test)
+add_dependencies(rpc-mgr-kerberized-test rpc_test_proto)
+target_link_libraries(rpc-mgr-kerberized-test rpc_test_proto)
+target_link_libraries(rpc-mgr-kerberized-test security-test-for-impala)
+target_link_libraries(rpc-mgr-kerberized-test ${KRB5_REALM_OVERRIDE})
 
 add_library(rpc_test_proto ${RPC_TEST_PROTO_SRCS})
 add_dependencies(rpc_test_proto rpc_test_proto_tgt krpc)

http://git-wip-us.apache.org/repos/asf/impala/blob/d84657ba/be/src/rpc/rpc-mgr-kerberized-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-kerberized-test.cc b/be/src/rpc/rpc-mgr-kerberized-test.cc
new file mode 100644
index 0000000..57ed3eb
--- /dev/null
+++ b/be/src/rpc/rpc-mgr-kerberized-test.cc
@@ -0,0 +1,89 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "rpc/rpc-mgr-test-base.h"
+
+DECLARE_string(ssl_client_ca_certificate);
+DECLARE_string(ssl_server_certificate);
+DECLARE_string(ssl_private_key);
+
+namespace impala {
+
+static int kdc_port = GetServerPort();
+
+class RpcMgrKerberizedTest :
+    public RpcMgrTestBase<testing::TestWithParam<KerberosSwitch> > {
+  virtual void SetUp() {
+    IpAddr ip;
+    ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+    string spn = Substitute("impala-test/$0", ip);
+
+    kdc_wrapper_.reset(new MiniKdcWrapper(
+        std::move(spn), "KRBTEST.COM", "24h", "7d", kdc_port));
+    DCHECK(kdc_wrapper_.get() != nullptr);
+
+    ASSERT_OK(kdc_wrapper_->SetupAndStartMiniKDC(GetParam()));
+    ASSERT_OK(InitAuth(CURRENT_EXECUTABLE_PATH));
+
+    RpcMgrTestBase::SetUp();
+  }
+
+  virtual void TearDown() {
+    ASSERT_OK(kdc_wrapper_->TearDownMiniKDC(GetParam()));
+    RpcMgrTestBase::TearDown();
+  }
+
+ private:
+  boost::scoped_ptr<MiniKdcWrapper> kdc_wrapper_;
+};
+
+INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
+                        RpcMgrKerberizedTest,
+                        ::testing::Values(USE_KUDU_KERBEROS,
+                                          USE_IMPALA_KERBEROS));
+
+TEST_P(RpcMgrKerberizedTest, MultipleServicesTls) {
+  // TODO: We're starting a seperate RpcMgr here instead of configuring
+  // RpcTestBase::rpc_mgr_ to use TLS. To use RpcTestBase::rpc_mgr_, we need to introduce
+  // new gtest params to turn on TLS which needs to be a coordinated change across
+  // rpc-mgr-test and thrift-server-test.
+  RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
+  TNetworkAddress tls_krpc_address;
+  IpAddr ip;
+  ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+
+  int32_t tls_service_port = FindUnusedEphemeralPort(nullptr);
+  tls_krpc_address = MakeNetworkAddress(ip, tls_service_port);
+
+  // Enable TLS.
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT);
+  ASSERT_OK(tls_rpc_mgr.Init());
+
+  ASSERT_OK(RunMultipleServicesTestTemplate(this, &tls_rpc_mgr, tls_krpc_address));
+  tls_rpc_mgr.Shutdown();
+}
+
+} // namespace impala
+
+int main(int argc, char** argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+  impala::InitCommonRuntime(argc, argv, false, impala::TestInfo::BE_TEST);
+
+  // Fill in the path of the current binary for use by the tests.
+  CURRENT_EXECUTABLE_PATH = argv[0];
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/d84657ba/be/src/rpc/rpc-mgr-test-base.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-test-base.h b/be/src/rpc/rpc-mgr-test-base.h
new file mode 100644
index 0000000..43b6d83
--- /dev/null
+++ b/be/src/rpc/rpc-mgr-test-base.h
@@ -0,0 +1,269 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "rpc/rpc-mgr.inline.h"
+
+#include "common/init.h"
+#include "exec/kudu-util.h"
+#include "kudu/rpc/rpc_context.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
+#include "rpc/auth-provider.h"
+#include "runtime/mem-tracker.h"
+#include "testutil/gtest-util.h"
+#include "testutil/mini-kdc-wrapper.h"
+#include "testutil/scoped-flag-setter.h"
+#include "util/counting-barrier.h"
+#include "util/network-util.h"
+#include "util/openssl-util.h"
+#include "util/test-info.h"
+
+#include "gen-cpp/rpc_test.proxy.h"
+#include "gen-cpp/rpc_test.service.h"
+
+#include "common/names.h"
+
+using kudu::rpc::ServiceIf;
+using kudu::rpc::RpcController;
+using kudu::rpc::RpcContext;
+using kudu::rpc::RpcSidecar;
+using kudu::Slice;
+
+using namespace std;
+
+DECLARE_int32(num_reactor_threads);
+DECLARE_int32(num_acceptor_threads);
+DECLARE_string(hostname);
+
+DECLARE_string(ssl_client_ca_certificate);
+DECLARE_string(ssl_server_certificate);
+DECLARE_string(ssl_private_key);
+DECLARE_string(ssl_private_key_password_cmd);
+DECLARE_string(ssl_cipher_list);
+
+// The path of the current executable file that is required for passing into the SASL
+// library as the 'application name'.
+static string CURRENT_EXECUTABLE_PATH;
+
+namespace impala {
+
+static int32_t SERVICE_PORT = FindUnusedEphemeralPort(nullptr);
+
+int GetServerPort() {
+  int port = FindUnusedEphemeralPort(nullptr);
+  EXPECT_FALSE(port == -1);
+  return port;
+}
+
+const static string IMPALA_HOME(getenv("IMPALA_HOME"));
+const string& SERVER_CERT =
+    Substitute("$0/be/src/testutil/server-cert.pem", IMPALA_HOME);
+const string& PRIVATE_KEY =
+    Substitute("$0/be/src/testutil/server-key.pem", IMPALA_HOME);
+const string& BAD_SERVER_CERT =
+    Substitute("$0/be/src/testutil/bad-cert.pem", IMPALA_HOME);
+const string& BAD_PRIVATE_KEY =
+    Substitute("$0/be/src/testutil/bad-key.pem", IMPALA_HOME);
+const string& PASSWORD_PROTECTED_PRIVATE_KEY =
+    Substitute("$0/be/src/testutil/server-key-password.pem", IMPALA_HOME);
+
+/// Use this class to set the appropriate required TLS flags for the duration of the
+/// lifetime of the object.
+/// It is assumed that the flags always hold empty values by default.
+class ScopedSetTlsFlags {
+ public:
+  ScopedSetTlsFlags(const string& cert, const string& pkey, const string& ca_cert,
+      const string& pkey_passwd = "", const string& ciphers = "") {
+    FLAGS_ssl_server_certificate = cert;
+    FLAGS_ssl_private_key = pkey;
+    FLAGS_ssl_client_ca_certificate = ca_cert;
+    FLAGS_ssl_private_key_password_cmd = pkey_passwd;
+    FLAGS_ssl_cipher_list = ciphers;
+  }
+
+  ~ScopedSetTlsFlags() {
+    FLAGS_ssl_server_certificate = "";
+    FLAGS_ssl_private_key = "";
+    FLAGS_ssl_client_ca_certificate = "";
+    FLAGS_ssl_private_key_password_cmd = "";
+    FLAGS_ssl_cipher_list = "";
+  }
+};
+
+// Only use TLSv1.0 compatible ciphers, as tests might run on machines with only TLSv1.0
+// support.
+const string TLS1_0_COMPATIBLE_CIPHER = "RC4-SHA";
+const string TLS1_0_COMPATIBLE_CIPHER_2 = "RC4-MD5";
+
+#define PAYLOAD_SIZE (4096)
+
+template <class T> class RpcMgrTestBase : public T {
+ public:
+  // Utility function to initialize the parameter for ScanMem RPC.
+  // Picks a random value and fills 'payload_' with it. Adds 'payload_' as a sidecar
+  // to 'controller'. Also sets up 'request' with the random value and index of the
+  // sidecar.
+  void SetupScanMemRequest(ScanMemRequestPB* request, RpcController* controller) {
+    int32_t pattern = random();
+    for (int i = 0; i < PAYLOAD_SIZE / sizeof(int32_t); ++i) payload_[i] = pattern;
+    int idx;
+    Slice slice(reinterpret_cast<const uint8_t*>(payload_), PAYLOAD_SIZE);
+    controller->AddOutboundSidecar(RpcSidecar::FromSlice(slice), &idx);
+    request->set_pattern(pattern);
+    request->set_sidecar_idx(idx);
+  }
+
+  MemTracker* service_tracker() { return &service_tracker_; }
+
+ protected:
+  TNetworkAddress krpc_address_;
+  RpcMgr rpc_mgr_;
+
+  virtual void SetUp() {
+    IpAddr ip;
+    ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
+    krpc_address_ = MakeNetworkAddress(ip, SERVICE_PORT);
+    ASSERT_OK(rpc_mgr_.Init());
+  }
+
+  virtual void TearDown() {
+    rpc_mgr_.Shutdown();
+  }
+
+ private:
+  int32_t payload_[PAYLOAD_SIZE];
+  MemTracker service_tracker_;
+};
+
+typedef std::function<void(RpcContext*)> ServiceCB;
+
+class PingServiceImpl : public PingServiceIf {
+ public:
+  // 'cb' is a callback used by tests to inject custom behaviour into the RPC handler.
+  PingServiceImpl(const scoped_refptr<kudu::MetricEntity>& entity,
+      const scoped_refptr<kudu::rpc::ResultTracker> tracker, MemTracker* mem_tracker,
+      ServiceCB cb = [](RpcContext* ctx) { ctx->RespondSuccess(); })
+    : PingServiceIf(entity, tracker), mem_tracker_(mem_tracker), cb_(cb) {}
+
+  virtual void Ping(
+      const PingRequestPB* request, PingResponsePB* response, RpcContext* context) {
+    response->set_int_response(42);
+    // Incoming requests will already be tracked and we need to release the memory.
+    mem_tracker_->Release(context->GetTransferSize());
+    cb_(context);
+  }
+
+ private:
+  MemTracker* mem_tracker_;
+  ServiceCB cb_;
+};
+
+class ScanMemServiceImpl : public ScanMemServiceIf {
+ public:
+  ScanMemServiceImpl(const scoped_refptr<kudu::MetricEntity>& entity,
+      const scoped_refptr<kudu::rpc::ResultTracker> tracker, MemTracker* mem_tracker)
+    : ScanMemServiceIf(entity, tracker), mem_tracker_(mem_tracker) {
+  }
+
+  // The request comes with an int 'pattern' and a payload of int array sent with
+  // sidecar. Scan the array to make sure every element matches 'pattern'.
+  virtual void ScanMem(const ScanMemRequestPB* request, ScanMemResponsePB* response,
+      RpcContext* context) {
+    int32_t pattern = request->pattern();
+    Slice payload;
+    ASSERT_OK(
+        FromKuduStatus(context->GetInboundSidecar(request->sidecar_idx(), &payload)));
+    ASSERT_EQ(payload.size() % sizeof(int32_t), 0);
+
+    const int32_t* v = reinterpret_cast<const int32_t*>(payload.data());
+    for (int i = 0; i < payload.size() / sizeof(int32_t); ++i) {
+      int32_t val = v[i];
+      if (val != pattern) {
+        // Incoming requests will already be tracked and we need to release the memory.
+        mem_tracker_->Release(context->GetTransferSize());
+        context->RespondFailure(kudu::Status::Corruption(
+            Substitute("Expecting $1; Found $2", pattern, val)));
+        return;
+      }
+    }
+    // Incoming requests will already be tracked and we need to release the memory.
+    mem_tracker_->Release(context->GetTransferSize());
+    context->RespondSuccess();
+  }
+
+ private:
+  MemTracker* mem_tracker_;
+
+};
+
+template <class T>
+Status RunMultipleServicesTestTemplate(RpcMgrTestBase<T>* test_base,
+    RpcMgr* rpc_mgr, const TNetworkAddress& krpc_address) {
+  MemTracker* mem_tracker = test_base->service_tracker();
+  // Test that a service can be started, and will respond to requests.
+  unique_ptr<ServiceIf> ping_impl(new PingServiceImpl(rpc_mgr->metric_entity(),
+      rpc_mgr->result_tracker(), mem_tracker));
+  RETURN_IF_ERROR(rpc_mgr->RegisterService(10, 10, move(ping_impl), mem_tracker));
+
+  // Test that a second service, that verifies the RPC payload is not corrupted,
+  // can be started.
+  unique_ptr<ServiceIf> scan_mem_impl(new ScanMemServiceImpl(rpc_mgr->metric_entity(),
+      rpc_mgr->result_tracker(), mem_tracker));
+  RETURN_IF_ERROR(rpc_mgr->RegisterService(10, 10, move(scan_mem_impl), mem_tracker));
+
+  FLAGS_num_acceptor_threads = 2;
+  FLAGS_num_reactor_threads = 10;
+  RETURN_IF_ERROR(rpc_mgr->StartServices(krpc_address));
+
+  unique_ptr<PingServiceProxy> ping_proxy;
+  RETURN_IF_ERROR(rpc_mgr->GetProxy<PingServiceProxy>(krpc_address, &ping_proxy));
+
+  unique_ptr<ScanMemServiceProxy> scan_mem_proxy;
+  RETURN_IF_ERROR(rpc_mgr->GetProxy<ScanMemServiceProxy>(krpc_address, &scan_mem_proxy));
+
+  RpcController controller;
+  srand(0);
+  // Randomly invoke either services to make sure a RpcMgr can host multiple
+  // services at the same time.
+  for (int i = 0; i < 100; ++i) {
+    controller.Reset();
+    if (random() % 2 == 0) {
+      PingRequestPB request;
+      PingResponsePB response;
+      KUDU_RETURN_IF_ERROR(ping_proxy->Ping(request, &response, &controller),
+          "unable to execute Ping() RPC.");
+      if (response.int_response() != 42) {
+          return Status(Substitute(
+              "Ping() failed. Incorrect response. Expected: 42; Got: $0",
+                  response.int_response()));
+      }
+    } else {
+      ScanMemRequestPB request;
+      ScanMemResponsePB response;
+      test_base->SetupScanMemRequest(&request, &controller);
+      KUDU_RETURN_IF_ERROR(scan_mem_proxy->ScanMem(request, &response, &controller),
+          "unable to execute ScanMem() RPC.");
+    }
+  }
+
+  return Status::OK();
+}
+
+} // namespace impala

http://git-wip-us.apache.org/repos/asf/impala/blob/d84657ba/be/src/rpc/rpc-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/rpc-mgr-test.cc b/be/src/rpc/rpc-mgr-test.cc
index c525148..4c4b100 100644
--- a/be/src/rpc/rpc-mgr-test.cc
+++ b/be/src/rpc/rpc-mgr-test.cc
@@ -15,124 +15,19 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "rpc/rpc-mgr.inline.h"
-
-#include "common/init.h"
-#include "exec/kudu-util.h"
-#include "kudu/rpc/rpc_context.h"
-#include "kudu/rpc/rpc_controller.h"
-#include "kudu/rpc/rpc_header.pb.h"
-#include "kudu/rpc/rpc_sidecar.h"
-#include "kudu/util/monotime.h"
-#include "kudu/util/status.h"
-#include "rpc/auth-provider.h"
-#include "runtime/mem-tracker.h"
-#include "testutil/gtest-util.h"
-#include "testutil/mini-kdc-wrapper.h"
-#include "testutil/scoped-flag-setter.h"
-#include "util/counting-barrier.h"
-#include "util/network-util.h"
-#include "util/openssl-util.h"
-#include "util/test-info.h"
-
-#include "gen-cpp/rpc_test.proxy.h"
-#include "gen-cpp/rpc_test.service.h"
-
-#include "common/names.h"
-
-using kudu::rpc::ErrorStatusPB;
+#include "rpc/rpc-mgr-test-base.h"
+
 using kudu::rpc::ServiceIf;
 using kudu::rpc::RpcController;
 using kudu::rpc::RpcContext;
-using kudu::rpc::RpcSidecar;
 using kudu::MonoDelta;
-using kudu::Slice;
-
-using namespace std;
 
 DECLARE_int32(num_reactor_threads);
 DECLARE_int32(num_acceptor_threads);
 DECLARE_string(hostname);
 
-DECLARE_string(ssl_client_ca_certificate);
-DECLARE_string(ssl_server_certificate);
-DECLARE_string(ssl_private_key);
-DECLARE_string(ssl_private_key_password_cmd);
-DECLARE_string(ssl_cipher_list);
-
-// The path of the current executable file that is required for passing into the SASL
-// library as the 'application name'.
-static string CURRENT_EXECUTABLE_PATH;
-
 namespace impala {
 
-static int32_t SERVICE_PORT = FindUnusedEphemeralPort(nullptr);
-
-int GetServerPort() {
-  int port = FindUnusedEphemeralPort(nullptr);
-  EXPECT_FALSE(port == -1);
-  return port;
-}
-
-static int kdc_port = GetServerPort();
-
-const static string IMPALA_HOME(getenv("IMPALA_HOME"));
-const string& SERVER_CERT =
-    Substitute("$0/be/src/testutil/server-cert.pem", IMPALA_HOME);
-const string& PRIVATE_KEY =
-    Substitute("$0/be/src/testutil/server-key.pem", IMPALA_HOME);
-const string& BAD_SERVER_CERT =
-    Substitute("$0/be/src/testutil/bad-cert.pem", IMPALA_HOME);
-const string& BAD_PRIVATE_KEY =
-    Substitute("$0/be/src/testutil/bad-key.pem", IMPALA_HOME);
-const string& PASSWORD_PROTECTED_PRIVATE_KEY =
-    Substitute("$0/be/src/testutil/server-key-password.pem", IMPALA_HOME);
-
-// Only use TLSv1.0 compatible ciphers, as tests might run on machines with only TLSv1.0
-// support.
-const string TLS1_0_COMPATIBLE_CIPHER = "RC4-SHA";
-const string TLS1_0_COMPATIBLE_CIPHER_2 = "RC4-MD5";
-
-#define PAYLOAD_SIZE (4096)
-
-template <class T> class RpcMgrTestBase : public T {
- public:
-  // Utility function to initialize the parameter for ScanMem RPC.
-  // Picks a random value and fills 'payload_' with it. Adds 'payload_' as a sidecar
-  // to 'controller'. Also sets up 'request' with the random value and index of the
-  // sidecar.
-  void SetupScanMemRequest(ScanMemRequestPB* request, RpcController* controller) {
-    int32_t pattern = random();
-    for (int i = 0; i < PAYLOAD_SIZE / sizeof(int32_t); ++i) payload_[i] = pattern;
-    int idx;
-    Slice slice(reinterpret_cast<const uint8_t*>(payload_), PAYLOAD_SIZE);
-    controller->AddOutboundSidecar(RpcSidecar::FromSlice(slice), &idx);
-    request->set_pattern(pattern);
-    request->set_sidecar_idx(idx);
-  }
-
-  MemTracker* service_tracker() { return &service_tracker_; }
-
- protected:
-  TNetworkAddress krpc_address_;
-  RpcMgr rpc_mgr_;
-
-  virtual void SetUp() {
-    IpAddr ip;
-    ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
-    krpc_address_ = MakeNetworkAddress(ip, SERVICE_PORT);
-    ASSERT_OK(rpc_mgr_.Init());
-  }
-
-  virtual void TearDown() {
-    rpc_mgr_.Shutdown();
-  }
-
- private:
-  int32_t payload_[PAYLOAD_SIZE];
-  MemTracker service_tracker_;
-};
-
 // For tests that do not require kerberized testing, we use RpcTest.
 class RpcMgrTest : public RpcMgrTestBase<testing::Test> {
   virtual void SetUp() {
@@ -144,157 +39,7 @@ class RpcMgrTest : public RpcMgrTestBase<testing::Test> {
   }
 };
 
-class RpcMgrKerberizedTest :
-    public RpcMgrTestBase<testing::TestWithParam<KerberosSwitch> > {
-  virtual void SetUp() {
-    IpAddr ip;
-    ASSERT_OK(HostnameToIpAddr(FLAGS_hostname, &ip));
-    string spn = Substitute("impala-test/$0", ip);
-
-    kdc_wrapper_.reset(new MiniKdcWrapper(
-        std::move(spn), "KRBTEST.COM", "24h", "7d", kdc_port));
-    DCHECK(kdc_wrapper_.get() != nullptr);
-
-    ASSERT_OK(kdc_wrapper_->SetupAndStartMiniKDC(GetParam()));
-    ASSERT_OK(InitAuth(CURRENT_EXECUTABLE_PATH));
-
-    RpcMgrTestBase::SetUp();
-  }
-
-  virtual void TearDown() {
-    ASSERT_OK(kdc_wrapper_->TearDownMiniKDC(GetParam()));
-    RpcMgrTestBase::TearDown();
-  }
-
- private:
-  boost::scoped_ptr<MiniKdcWrapper> kdc_wrapper_;
-};
-
-typedef std::function<void(RpcContext*)> ServiceCB;
-
-class PingServiceImpl : public PingServiceIf {
- public:
-  // 'cb' is a callback used by tests to inject custom behaviour into the RPC handler.
-  PingServiceImpl(const scoped_refptr<kudu::MetricEntity>& entity,
-      const scoped_refptr<kudu::rpc::ResultTracker> tracker, MemTracker* mem_tracker,
-      ServiceCB cb = [](RpcContext* ctx) { ctx->RespondSuccess(); })
-    : PingServiceIf(entity, tracker), mem_tracker_(mem_tracker), cb_(cb) {}
-
-  virtual void Ping(
-      const PingRequestPB* request, PingResponsePB* response, RpcContext* context) {
-    response->set_int_response(42);
-    // Incoming requests will already be tracked and we need to release the memory.
-    mem_tracker_->Release(context->GetTransferSize());
-    cb_(context);
-  }
-
- private:
-  MemTracker* mem_tracker_;
-  ServiceCB cb_;
-};
-
-class ScanMemServiceImpl : public ScanMemServiceIf {
- public:
-  ScanMemServiceImpl(const scoped_refptr<kudu::MetricEntity>& entity,
-      const scoped_refptr<kudu::rpc::ResultTracker> tracker, MemTracker* mem_tracker)
-    : ScanMemServiceIf(entity, tracker), mem_tracker_(mem_tracker) {
-  }
-
-  // The request comes with an int 'pattern' and a payload of int array sent with
-  // sidecar. Scan the array to make sure every element matches 'pattern'.
-  virtual void ScanMem(const ScanMemRequestPB* request, ScanMemResponsePB* response,
-      RpcContext* context) {
-    int32_t pattern = request->pattern();
-    Slice payload;
-    ASSERT_OK(
-        FromKuduStatus(context->GetInboundSidecar(request->sidecar_idx(), &payload)));
-    ASSERT_EQ(payload.size() % sizeof(int32_t), 0);
-
-    const int32_t* v = reinterpret_cast<const int32_t*>(payload.data());
-    for (int i = 0; i < payload.size() / sizeof(int32_t); ++i) {
-      int32_t val = v[i];
-      if (val != pattern) {
-        // Incoming requests will already be tracked and we need to release the memory.
-        mem_tracker_->Release(context->GetTransferSize());
-        context->RespondFailure(kudu::Status::Corruption(
-            Substitute("Expecting $1; Found $2", pattern, val)));
-        return;
-      }
-    }
-    // Incoming requests will already be tracked and we need to release the memory.
-    mem_tracker_->Release(context->GetTransferSize());
-    context->RespondSuccess();
-  }
-
- private:
-  MemTracker* mem_tracker_;
-};
-
-// TODO: USE_KUDU_KERBEROS and USE_IMPALA_KERBEROS are disabled due to IMPALA-6448.
-// Re-enable after fixing.
-INSTANTIATE_TEST_CASE_P(KerberosOnAndOff,
-                        RpcMgrKerberizedTest,
-                        ::testing::Values(KERBEROS_OFF));
-
-template <class T>
-Status RunMultipleServicesTestTemplate(RpcMgrTestBase<T>* test_base,
-    RpcMgr* rpc_mgr, const TNetworkAddress& krpc_address) {
-  MemTracker* mem_tracker = test_base->service_tracker();
-  // Test that a service can be started, and will respond to requests.
-  unique_ptr<ServiceIf> ping_impl(new PingServiceImpl(rpc_mgr->metric_entity(),
-      rpc_mgr->result_tracker(), mem_tracker));
-  RETURN_IF_ERROR(rpc_mgr->RegisterService(10, 10, move(ping_impl), mem_tracker));
-
-  // Test that a second service, that verifies the RPC payload is not corrupted,
-  // can be started.
-  unique_ptr<ServiceIf> scan_mem_impl(new ScanMemServiceImpl(rpc_mgr->metric_entity(),
-      rpc_mgr->result_tracker(), mem_tracker));
-  RETURN_IF_ERROR(rpc_mgr->RegisterService(10, 10, move(scan_mem_impl), mem_tracker));
-
-  FLAGS_num_acceptor_threads = 2;
-  FLAGS_num_reactor_threads = 10;
-  RETURN_IF_ERROR(rpc_mgr->StartServices(krpc_address));
-
-  unique_ptr<PingServiceProxy> ping_proxy;
-  RETURN_IF_ERROR(rpc_mgr->GetProxy<PingServiceProxy>(krpc_address, &ping_proxy));
-
-  unique_ptr<ScanMemServiceProxy> scan_mem_proxy;
-  RETURN_IF_ERROR(rpc_mgr->GetProxy<ScanMemServiceProxy>(krpc_address, &scan_mem_proxy));
-
-  RpcController controller;
-  srand(0);
-  // Randomly invoke either services to make sure a RpcMgr can host multiple
-  // services at the same time.
-  for (int i = 0; i < 100; ++i) {
-    controller.Reset();
-    if (random() % 2 == 0) {
-      PingRequestPB request;
-      PingResponsePB response;
-      KUDU_RETURN_IF_ERROR(ping_proxy->Ping(request, &response, &controller),
-          "unable to execute Ping() RPC.");
-      if (response.int_response() != 42) {
-          return Status(Substitute(
-              "Ping() failed. Incorrect response. Expected: 42; Got: $0",
-                  response.int_response()));
-      }
-    } else {
-      ScanMemRequestPB request;
-      ScanMemResponsePB response;
-      test_base->SetupScanMemRequest(&request, &controller);
-      KUDU_RETURN_IF_ERROR(scan_mem_proxy->ScanMem(request, &response, &controller),
-          "unable to execute ScanMem() RPC.");
-    }
-  }
-
-  return Status::OK();
-}
-
-
-TEST_F(RpcMgrTest, MultipleServices) {
-  ASSERT_OK(RunMultipleServicesTestTemplate(this, &rpc_mgr_, krpc_address_));
-}
-
-TEST_P(RpcMgrKerberizedTest, MultipleServicesTls) {
+TEST_F(RpcMgrTest, MultipleServicesTls) {
   // TODO: We're starting a seperate RpcMgr here instead of configuring
   // RpcTestBase::rpc_mgr_ to use TLS. To use RpcTestBase::rpc_mgr_, we need to introduce
   // new gtest params to turn on TLS which needs to be a coordinated change across
@@ -307,28 +52,20 @@ TEST_P(RpcMgrKerberizedTest, MultipleServicesTls) {
   int32_t tls_service_port = FindUnusedEphemeralPort(nullptr);
   tls_krpc_address = MakeNetworkAddress(ip, tls_service_port);
 
-  // Enable TLS.
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_private_key, PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT);
   ASSERT_OK(tls_rpc_mgr.Init());
 
   ASSERT_OK(RunMultipleServicesTestTemplate(this, &tls_rpc_mgr, tls_krpc_address));
   tls_rpc_mgr.Shutdown();
 }
 
+TEST_F(RpcMgrTest, MultipleServices) {
+  ASSERT_OK(RunMultipleServicesTestTemplate(this, &rpc_mgr_, krpc_address_));
+}
+
 // Test with a misconfigured TLS certificate and verify that an error is thrown.
 TEST_F(RpcMgrTest, BadCertificateTls) {
-
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_private_key, PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, "unknown");
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, "unknown");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   TNetworkAddress tls_krpc_address;
@@ -344,16 +81,8 @@ TEST_F(RpcMgrTest, BadCertificateTls) {
 
 // Test with a bad password command for the password protected private key.
 TEST_F(RpcMgrTest, BadPasswordTls) {
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(
-          &FLAGS_ssl_private_key, PASSWORD_PROTECTED_PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
-  auto password_cmd =
-      ScopedFlagSetter<string>::Make(
-          &FLAGS_ssl_private_key_password_cmd, "echo badpassword");
+  ScopedSetTlsFlags s(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY, SERVER_CERT,
+      "echo badpassword");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   TNetworkAddress tls_krpc_address;
@@ -369,16 +98,8 @@ TEST_F(RpcMgrTest, BadPasswordTls) {
 
 // Test with a correct password command for the password protected private key.
 TEST_F(RpcMgrTest, CorrectPasswordTls) {
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(
-          &FLAGS_ssl_private_key, PASSWORD_PROTECTED_PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
-  auto password_cmd =
-      ScopedFlagSetter<string>::Make(
-          &FLAGS_ssl_private_key_password_cmd, "echo password");
+  ScopedSetTlsFlags s(SERVER_CERT, PASSWORD_PROTECTED_PRIVATE_KEY, SERVER_CERT,
+      "echo password");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   TNetworkAddress tls_krpc_address;
@@ -395,14 +116,7 @@ TEST_F(RpcMgrTest, CorrectPasswordTls) {
 
 // Test with a bad TLS cipher and verify that an error is thrown.
 TEST_F(RpcMgrTest, BadCiphersTls) {
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_private_key, PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
-  auto cipher =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list, "not_a_cipher");
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", "not_a_cipher");
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   TNetworkAddress tls_krpc_address;
@@ -418,14 +132,8 @@ TEST_F(RpcMgrTest, BadCiphersTls) {
 
 // Test with a valid TLS cipher.
 TEST_F(RpcMgrTest, ValidCiphersTls) {
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_private_key, PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
-  auto cipher =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list, TLS1_0_COMPATIBLE_CIPHER);
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "",
+      TLS1_0_COMPATIBLE_CIPHER);
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   TNetworkAddress tls_krpc_address;
@@ -444,14 +152,7 @@ TEST_F(RpcMgrTest, ValidCiphersTls) {
 TEST_F(RpcMgrTest, ValidMultiCiphersTls) {
   const string cipher_list = Substitute("$0,$1", TLS1_0_COMPATIBLE_CIPHER,
       TLS1_0_COMPATIBLE_CIPHER_2);
-  auto cert_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_server_certificate, SERVER_CERT);
-  auto pkey_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_private_key, PRIVATE_KEY);
-  auto ca_flag =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_client_ca_certificate, SERVER_CERT);
-  auto cipher =
-      ScopedFlagSetter<string>::Make(&FLAGS_ssl_cipher_list, cipher_list);
+  ScopedSetTlsFlags s(SERVER_CERT, PRIVATE_KEY, SERVER_CERT, "", cipher_list);
 
   RpcMgr tls_rpc_mgr(IsInternalTlsConfigured());
   TNetworkAddress tls_krpc_address;


[13/21] impala git commit: IMPALA-6449: Use CLOCK_MONOTONIC in ConditionVariable

Posted by ta...@apache.org.
IMPALA-6449: Use CLOCK_MONOTONIC in ConditionVariable

ConditionVariable is a thin wrapper around pthread_cond_*.
Currently, pthread_cond_timedwait() uses the default attribute
CLOCK_REALTIME. This is susceptible to adjustment to the system
clock from various sources such as NTP and time may go backward.
This change fixes the problem by switching to using CLOCK_MONOTONIC
so time will be monotonic although the frequency of the clock ticks
may still be adjusted by NTP. Ideally, we should use CLOCK_MONOTONIC_RAW
but it's available only on Linux kernel 2.6.28 or latter. This change
also get rids of some usage of boost::get_system_time() which suffers
from the same problem.

Change-Id: I81611cfd5e7c5347203fe7fa6b0f615602257f87
Reviewed-on: http://gerrit.cloudera.org:8080/9158
Reviewed-by: Michael Ho <kw...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: ee74a6277dec51fd1cd32acfdfb7821174451c03
Parents: d747670
Author: Michael Ho <kw...@cloudera.com>
Authored: Mon Jan 29 18:07:28 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/rpc/thrift-server.cc               |  4 +--
 be/src/runtime/fragment-instance-state.cc |  7 ++---
 be/src/service/impala-server.cc           |  2 +-
 be/src/util/blocking-queue.h              |  6 ++--
 be/src/util/condition-variable.h          | 40 +++++++++++++-------------
 be/src/util/promise.h                     |  8 ++----
 be/src/util/time.h                        | 13 +++++++++
 7 files changed, 44 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/rpc/thrift-server.cc
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-server.cc b/be/src/rpc/thrift-server.cc
index ded710e..48fb1b9 100644
--- a/be/src/rpc/thrift-server.cc
+++ b/be/src/rpc/thrift-server.cc
@@ -165,8 +165,8 @@ Status ThriftServer::ThriftServerEventProcessor::StartAndWaitForServer() {
       &ThriftServer::ThriftServerEventProcessor::Supervise, this,
       &thrift_server_->server_thread_));
 
-  system_time deadline = get_system_time() +
-      posix_time::milliseconds(ThriftServer::ThriftServerEventProcessor::TIMEOUT_MS);
+  timespec deadline;
+  TimeFromNowMillis(ThriftServer::ThriftServerEventProcessor::TIMEOUT_MS, &deadline);
 
   // Loop protects against spurious wakeup. Locks provide necessary fences to ensure
   // visibility.

http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/runtime/fragment-instance-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/fragment-instance-state.cc b/be/src/runtime/fragment-instance-state.cc
index 16b4a7e..ad9e99e 100644
--- a/be/src/runtime/fragment-instance-state.cc
+++ b/be/src/runtime/fragment-instance-state.cc
@@ -341,19 +341,16 @@ void FragmentInstanceState::ReportProfileThread() {
   // updates at once so its better for contention as well as smoother progress
   // reporting.
   int report_fragment_offset = rand() % FLAGS_status_report_interval;
-  boost::posix_time::seconds wait_duration(report_fragment_offset);
   // We don't want to wait longer than it takes to run the entire fragment.
-  stop_report_thread_cv_.WaitFor(l, wait_duration);
+  stop_report_thread_cv_.WaitFor(l, report_fragment_offset * MICROS_PER_SEC);
 
   while (report_thread_active_) {
-    boost::posix_time::seconds loop_wait_duration(FLAGS_status_report_interval);
-
     // timed_wait can return because the timeout occurred or the condition variable
     // was signaled.  We can't rely on its return value to distinguish between the
     // two cases (e.g. there is a race here where the wait timed out but before grabbing
     // the lock, the condition variable was signaled).  Instead, we will use an external
     // flag, report_thread_active_, to coordinate this.
-    stop_report_thread_cv_.WaitFor(l, loop_wait_duration);
+    stop_report_thread_cv_.WaitFor(l, FLAGS_status_report_interval * MICROS_PER_SEC);
 
     if (!report_thread_active_) break;
     SendReport(false, Status::OK());

http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 0c5f75b..cf5f5fb 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -1772,7 +1772,7 @@ void ImpalaServer::UnregisterSessionTimeout(int32_t session_timeout) {
         session_timeout_cv_.Wait(timeout_lock);
       } else {
         // Sleep for a second before checking whether an active session can be expired.
-        session_timeout_cv_.WaitFor(timeout_lock, seconds(1));
+        session_timeout_cv_.WaitFor(timeout_lock, MICROS_PER_SEC);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/util/blocking-queue.h
----------------------------------------------------------------------
diff --git a/be/src/util/blocking-queue.h b/be/src/util/blocking-queue.h
index a4b1b8f..1dd90d5 100644
--- a/be/src/util/blocking-queue.h
+++ b/be/src/util/blocking-queue.h
@@ -138,13 +138,13 @@ class BlockingQueue : public CacheLineAligned {
   bool BlockingPutWithTimeout(V&& val, int64_t timeout_micros) {
     MonotonicStopWatch timer;
     boost::unique_lock<boost::mutex> write_lock(put_lock_);
-    boost::system_time wtime = boost::get_system_time() +
-        boost::posix_time::microseconds(timeout_micros);
+    timespec abs_time;
+    TimeFromNowMicros(timeout_micros, &abs_time);
     bool notified = true;
     while (SizeLocked(write_lock) >= max_elements_ && !shutdown_ && notified) {
       timer.Start();
       // Wait until we're notified or until the timeout expires.
-      notified = put_cv_.WaitUntil(write_lock, wtime);
+      notified = put_cv_.WaitUntil(write_lock, abs_time);
       timer.Stop();
     }
     total_put_wait_time_ += timer.ElapsedTime();

http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/util/condition-variable.h
----------------------------------------------------------------------
diff --git a/be/src/util/condition-variable.h b/be/src/util/condition-variable.h
index c1a1e56..e463790 100644
--- a/be/src/util/condition-variable.h
+++ b/be/src/util/condition-variable.h
@@ -24,13 +24,23 @@
 #include <pthread.h>
 #include <unistd.h>
 
+#include "util/time.h"
+
 namespace impala {
 
 /// Simple wrapper around POSIX pthread condition variable. This has lower overhead than
 /// boost's implementation as it doesn't implement boost thread interruption.
 class ConditionVariable {
  public:
-  ConditionVariable() { pthread_cond_init(&cv_, NULL); }
+  ConditionVariable() {
+    pthread_condattr_t attrs;
+    int retval = pthread_condattr_init(&attrs);
+    DCHECK_EQ(0, retval);
+    pthread_condattr_setclock(&attrs, CLOCK_MONOTONIC);
+    retval = pthread_cond_init(&cv_, &attrs);
+    DCHECK_EQ(0, retval);
+    pthread_condattr_destroy(&attrs);
+  }
 
   ~ConditionVariable() { pthread_cond_destroy(&cv_); }
 
@@ -41,32 +51,22 @@ class ConditionVariable {
     pthread_cond_wait(&cv_, mutex);
   }
 
-  /// Wait until the condition variable is notified or 'timeout' has passed.
+  /// Wait until the condition variable is notified or 'abs_time' has passed.
   /// Returns true if the condition variable is notified before the absolute timeout
-  /// specified in 'timeout' has passed. Returns false otherwise.
-  bool WaitUntil(boost::unique_lock<boost::mutex>& lock,
-      const timespec& abs_time) {
+  /// specified in 'abs_time' has passed. Returns false otherwise.
+  bool WaitUntil(boost::unique_lock<boost::mutex>& lock, const timespec& abs_time) {
     DCHECK(lock.owns_lock());
     pthread_mutex_t* mutex = lock.mutex()->native_handle();
     return pthread_cond_timedwait(&cv_, mutex, &abs_time) == 0;
   }
 
-  /// Wait until the condition variable is notified or 'abs_time' has passed.
-  /// Returns true if the condition variable is notified before the absolute timeout
-  /// specified in 'abs_time' has passed. Returns false otherwise.
-  bool WaitUntil(boost::unique_lock<boost::mutex>& lock,
-      const boost::system_time& abs_time) {
-    return WaitUntil(lock, to_timespec(abs_time));
-  }
-
-  /// Wait until the condition variable is notified or have waited for the time
-  /// specified in 'wait_duration'.
-  /// Returns true if the condition variable is notified in time.
+  /// Wait until the condition variable is notified or 'duration_us' microseconds
+  /// have passed. Returns true if the condition variable is notified in time.
   /// Returns false otherwise.
-  template <typename duration_type>
-  bool WaitFor(boost::unique_lock<boost::mutex>& lock,
-      const duration_type& wait_duration) {
-    return WaitUntil(lock, to_timespec(boost::get_system_time() + wait_duration));
+  bool WaitFor(boost::unique_lock<boost::mutex>& lock, int64_t duration_us) {
+    timespec deadline;
+    TimeFromNowMicros(duration_us, &deadline);
+    return WaitUntil(lock, deadline);
   }
 
   /// Notify a single waiter on this condition variable.

http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/util/promise.h
----------------------------------------------------------------------
diff --git a/be/src/util/promise.h b/be/src/util/promise.h
index 5de2d13..c93d9f2 100644
--- a/be/src/util/promise.h
+++ b/be/src/util/promise.h
@@ -77,17 +77,15 @@ class Promise {
   /// timed_out: Indicates whether Get() returned due to timeout. Must be non-NULL.
   const T& Get(int64_t timeout_millis, bool* timed_out) {
     DCHECK_GT(timeout_millis, 0);
-    int64_t timeout_micros = timeout_millis * 1000;
+    int64_t timeout_micros = timeout_millis * MICROS_PER_MILLI;
     DCHECK(timed_out != NULL);
     boost::unique_lock<boost::mutex> l(val_lock_);
     int64_t start;
     int64_t now;
     now = start = MonotonicMicros();
     while (!val_is_set_ && (now - start) < timeout_micros) {
-      boost::posix_time::microseconds wait_time =
-          boost::posix_time::microseconds(std::max<int64_t>(
-              1, timeout_micros - (now - start)));
-      val_set_cond_.WaitFor(l, wait_time);
+      int64_t wait_time_micros = std::max<int64_t>(1, timeout_micros - (now - start));
+      val_set_cond_.WaitFor(l, wait_time_micros);
       now = MonotonicMicros();
     }
     *timed_out = !val_is_set_;

http://git-wip-us.apache.org/repos/asf/impala/blob/ee74a627/be/src/util/time.h
----------------------------------------------------------------------
diff --git a/be/src/util/time.h b/be/src/util/time.h
index cef14c8..64dbf9c 100644
--- a/be/src/util/time.h
+++ b/be/src/util/time.h
@@ -57,6 +57,19 @@ inline int64_t UnixMillis() {
   return GetCurrentTimeMicros() / MICROS_PER_MILLI;
 }
 
+/// Return the time 'time_us' microseconds away from now in 'abs_time'.
+inline void TimeFromNowMicros(int64_t time_us, timespec* abs_time) {
+  clock_gettime(CLOCK_MONOTONIC, abs_time);
+  abs_time->tv_nsec += (time_us % MICROS_PER_SEC) * NANOS_PER_MICRO;
+  abs_time->tv_sec += time_us / MICROS_PER_SEC + abs_time->tv_nsec / NANOS_PER_SEC;
+  abs_time->tv_nsec %= NANOS_PER_SEC;
+}
+
+/// Return the time 'time_ms' milliseconds away from now in 'abs_time'.
+inline void TimeFromNowMillis(int64_t time_ms, timespec* abs_time) {
+  TimeFromNowMicros(time_ms * MICROS_PER_MILLI, abs_time);
+}
+
 /// Returns the number of microseconds that have passed since the Unix epoch. This is
 /// affected by manual changes to the system clock but is more suitable for use across
 /// a cluster. For more accurate timings on the local host use the monotonic functions


[07/21] impala git commit: IMPALA-5990: End-to-end compression of metadata

Posted by ta...@apache.org.
IMPALA-5990: End-to-end compression of metadata

Currently the catalog data is compressed in the statestore, but
uncompressed when passed between FE and BE. It results in a ~2GB limit
on the metadata. IMPALA-3499 introduced a workaround in the impalad but
there isn't one in the catalogd. This patch aims to increase the size
limit for statestore updates, reduce the copying of the metadata and
reduce the memory footprint. With this patch, the catalog objects are
passed and (de)compressed between FE and BE one at a time. The new
limits are:
- A single catalog object cannot be larger than ~2GB.
- A statestore catalog update cannot be larger than ~4GB. It is
  compressed size if FLAGS_compact_catalog_topic is true.
The behavior of the catalog op executer is not changed. The data is not
compressed and the size limit is still 2GB.

Testing: Ran existing tests. A test for compressing and decompressing
catalog objects is added. Manually tested with a 1.95GB catalog object
and a 3.90 GB uncompressed statestore update.

Change-Id: I3a8819cad734b3a416eef6c954e55b73cc6023ae
Reviewed-on: http://gerrit.cloudera.org:8080/8825
Reviewed-by: Tianyi Wang <tw...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: a2431638dc7ade13e99e5cb3db7f2f65888c45b9
Parents: f258a91
Author: Tianyi Wang <tw...@cloudera.com>
Authored: Tue Jan 16 16:37:29 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:52 2018 +0000

----------------------------------------------------------------------
 be/src/catalog/CMakeLists.txt                   |   2 +
 be/src/catalog/catalog-server.cc                |  78 +++----
 be/src/catalog/catalog-server.h                 |  19 +-
 be/src/catalog/catalog-util-test.cc             |  49 ++++
 be/src/catalog/catalog-util.cc                  | 164 ++++++++-----
 be/src/catalog/catalog-util.h                   |  93 ++++++--
 be/src/catalog/catalog.cc                       |   4 +-
 be/src/catalog/catalog.h                        |   9 +-
 be/src/rpc/thrift-util.h                        |  24 +-
 be/src/service/fe-support.cc                    | 102 ++++++++-
 be/src/service/frontend.cc                      |   6 +-
 be/src/service/frontend.h                       |   9 +-
 be/src/service/impala-server.cc                 | 185 +++------------
 be/src/service/impalad-main.cc                  |   2 +
 be/src/util/jni-util.cc                         |  21 +-
 be/src/util/jni-util.h                          |  54 +++--
 common/thrift/CatalogInternalService.thrift     |  16 +-
 common/thrift/Frontend.thrift                   |  22 +-
 .../java/org/apache/impala/catalog/Catalog.java |   2 +
 .../impala/catalog/CatalogServiceCatalog.java   | 229 +++++++++++--------
 .../apache/impala/catalog/ImpaladCatalog.java   | 145 +++++++-----
 .../org/apache/impala/service/FeSupport.java    |  31 ++-
 .../org/apache/impala/service/Frontend.java     |   3 +-
 .../org/apache/impala/service/JniCatalog.java   |  14 +-
 .../org/apache/impala/service/JniFrontend.java  |  29 +--
 .../org/apache/impala/util/TByteBuffer.java     |  60 +++++
 26 files changed, 812 insertions(+), 560 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/catalog/CMakeLists.txt b/be/src/catalog/CMakeLists.txt
index 7debb91..35cccea 100644
--- a/be/src/catalog/CMakeLists.txt
+++ b/be/src/catalog/CMakeLists.txt
@@ -25,3 +25,5 @@ add_library(Catalog
   catalogd-main.cc
 )
 add_dependencies(Catalog gen-deps)
+
+ADD_BE_TEST(catalog-util-test)

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog-server.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-server.cc b/be/src/catalog/catalog-server.cc
index 4bf26c0..2f4bcbe 100644
--- a/be/src/catalog/catalog-server.cc
+++ b/be/src/catalog/catalog-server.cc
@@ -238,14 +238,10 @@ void CatalogServer::UpdateCatalogTopicCallback(
     LOG_EVERY_N(INFO, 300) << "Catalog Version: " << catalog_objects_max_version_
                            << " Last Catalog Version: " << last_sent_catalog_version_;
 
-    for (const TTopicItem& catalog_object: pending_topic_updates_) {
-      if (subscriber_topic_updates->size() == 0) {
-        subscriber_topic_updates->push_back(TTopicDelta());
-        subscriber_topic_updates->back().topic_name = IMPALA_CATALOG_TOPIC;
-      }
-      TTopicDelta& update = subscriber_topic_updates->back();
-      update.topic_entries.push_back(catalog_object);
-    }
+    subscriber_topic_updates->emplace_back();
+    TTopicDelta& update = subscriber_topic_updates->back();
+    update.topic_name = IMPALA_CATALOG_TOPIC;
+    update.topic_entries = std::move(pending_topic_updates_);
 
     // Update the new catalog version and the set of known catalog objects.
     last_sent_catalog_version_ = catalog_objects_max_version_;
@@ -281,19 +277,13 @@ void CatalogServer::UpdateCatalogTopicCallback(
     } else if (current_catalog_version != last_sent_catalog_version_) {
       // If there has been a change since the last time the catalog was queried,
       // call into the Catalog to find out what has changed.
-      TGetCatalogDeltaResponse catalog_objects;
-      status = catalog_->GetCatalogDelta(last_sent_catalog_version_, &catalog_objects);
+      TGetCatalogDeltaResponse resp;
+      status = catalog_->GetCatalogDelta(this, last_sent_catalog_version_, &resp);
       if (!status.ok()) {
         LOG(ERROR) << status.GetDetail();
       } else {
-        // Use the catalog objects to build a topic update list. These include
-        // objects added to the catalog, 'updated_objects', and objects deleted
-        // from the catalog, 'deleted_objects'. The order in which we process
-        // these two disjoint sets of catalog objects does not matter.
-        BuildTopicUpdates(catalog_objects.updated_objects, false);
-        BuildTopicUpdates(catalog_objects.deleted_objects, true);
         catalog_objects_min_version_ = last_sent_catalog_version_;
-        catalog_objects_max_version_ = catalog_objects.max_catalog_version;
+        catalog_objects_max_version_ = resp.max_catalog_version;
       }
     }
 
@@ -302,37 +292,6 @@ void CatalogServer::UpdateCatalogTopicCallback(
   }
 }
 
-void CatalogServer::BuildTopicUpdates(const vector<TCatalogObject>& catalog_objects,
-    bool topic_deletions) {
-  for (const TCatalogObject& catalog_object: catalog_objects) {
-    DCHECK_GT(catalog_object.catalog_version, last_sent_catalog_version_);
-    const string& entry_key = TCatalogObjectToEntryKey(catalog_object);
-    if (entry_key.empty()) {
-      LOG_EVERY_N(WARNING, 60) << "Unable to build topic entry key for TCatalogObject: "
-                               << ThriftDebugString(catalog_object);
-    }
-    pending_topic_updates_.push_back(TTopicItem());
-    TTopicItem& item = pending_topic_updates_.back();
-    item.key = entry_key;
-    item.deleted = topic_deletions;
-    Status status = thrift_serializer_.Serialize(&catalog_object, &item.value);
-    if (!status.ok()) {
-      LOG(ERROR) << "Error serializing topic value: " << status.GetDetail();
-      pending_topic_updates_.pop_back();
-      continue;
-    }
-    if (FLAGS_compact_catalog_topic) {
-      status = CompressCatalogObject(&item.value);
-      if (!status.ok()) {
-        LOG(ERROR) << "Error compressing catalog object: " << status.GetDetail();
-        pending_topic_updates_.pop_back();
-      }
-    }
-    VLOG(1) << "Publishing " << (topic_deletions ? "deletion " : "update ")
-        << ": " << entry_key << "@" << catalog_object.catalog_version;
-  }
-}
-
 void CatalogServer::CatalogUrlCallback(const Webserver::ArgumentMap& args,
     Document* document) {
   GetCatalogUsage(document);
@@ -497,3 +456,26 @@ void CatalogServer::TableMetricsUrlCallback(const Webserver::ArgumentMap& args,
     document->AddMember("error", error, document->GetAllocator());
   }
 }
+
+bool CatalogServer::AddPendingTopicItem(std::string key, const uint8_t* item_data,
+    uint32_t size, bool deleted) {
+  pending_topic_updates_.emplace_back();
+  TTopicItem& item = pending_topic_updates_.back();
+  if (FLAGS_compact_catalog_topic) {
+    Status status = CompressCatalogObject(item_data, size, &item.value);
+    if (!status.ok()) {
+      pending_topic_updates_.pop_back();
+      LOG(ERROR) << "Error compressing topic item: " << status.GetDetail();
+      return false;
+    }
+  } else {
+    item.value.assign(reinterpret_cast<const char*>(item_data),
+        static_cast<size_t>(size));
+  }
+  item.key = std::move(key);
+  item.deleted = deleted;
+  VLOG(1) << "Publishing " << (deleted ? "deletion: " : "update: ") << item.key <<
+      " original size: " << size << (FLAGS_compact_catalog_topic ?
+      Substitute(" compressed size: $0", item.value.size()) : string());
+  return true;
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog-server.h
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-server.h b/be/src/catalog/catalog-server.h
index 0b6b220..a6a0c3f 100644
--- a/be/src/catalog/catalog-server.h
+++ b/be/src/catalog/catalog-server.h
@@ -74,6 +74,11 @@ class CatalogServer {
   }
   Catalog* catalog() const { return catalog_.get(); }
 
+  /// Add a topic item to pending_topic_updates_. Caller must hold catalog_lock_.
+  /// The return value is true if the operation succeeds and false otherwise.
+  bool AddPendingTopicItem(std::string key, const uint8_t* item_data, uint32_t size,
+      bool deleted);
+
  private:
   /// Thrift API implementation which proxies requests onto this CatalogService.
   boost::shared_ptr<CatalogServiceIf> thrift_iface_;
@@ -143,20 +148,6 @@ class CatalogServer {
   /// Also, explicitly releases free memory back to the OS after each complete iteration.
   [[noreturn]] void GatherCatalogUpdatesThread();
 
-  /// Builds the next topic update to send based on what items
-  /// have been added/changed/removed from the catalog since the last hearbeat. To do
-  /// this, it enumerates the given catalog objects returned looking for the objects that
-  /// have a catalog version that is > the catalog version sent with the last heartbeat.
-  /// 'topic_deletions' is true if 'catalog_objects' contain deleted catalog
-  /// objects.
-  ///
-  /// The key for each entry is a string composed of:
-  /// "TCatalogObjectType:<unique object name>". So for table foo.bar, the key would be
-  /// "TABLE:foo.bar". Encoding the object type information in the key ensures the keys
-  /// are unique. Must hold catalog_lock_ when calling this function.
-  void BuildTopicUpdates(const std::vector<TCatalogObject>& catalog_objects,
-      bool topic_deletions);
-
   /// Example output:
   /// "databases": [
   ///         {

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-util-test.cc b/be/src/catalog/catalog-util-test.cc
new file mode 100644
index 0000000..d37fc5c
--- /dev/null
+++ b/be/src/catalog/catalog-util-test.cc
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "catalog/catalog-util.h"
+#include "testutil/gtest-util.h"
+
+using namespace impala;
+using namespace std;
+
+void CompressAndDecompress(const std::string& input) {
+  string compressed;
+  string decompressed;
+  ASSERT_OK(CompressCatalogObject(reinterpret_cast<const uint8_t*>(input.data()),
+      static_cast<uint32_t>(input.size()), &compressed));
+  ASSERT_OK(DecompressCatalogObject(reinterpret_cast<const uint8_t*>(compressed.data()),
+      static_cast<uint32_t>(compressed.size()), &decompressed));
+  ASSERT_EQ(input.size(), decompressed.size());
+  ASSERT_EQ(input, decompressed);
+}
+
+
+TEST(CatalogUtil, TestCatalogCompression) {
+  CompressAndDecompress("");
+  CompressAndDecompress("deadbeef");
+  string large_string;
+  uint32_t large_string_size = 0x7E000000; // LZ4_MAX_INPUT_SIZE
+  large_string.reserve(large_string_size);
+  for (uint32_t i = 0; i < large_string_size; ++i) {
+    large_string.push_back(static_cast<char>(rand() % (1 + numeric_limits<char>::max())));
+  }
+  CompressAndDecompress(large_string);
+}
+
+IMPALA_TEST_MAIN();
+

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog-util.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-util.cc b/be/src/catalog/catalog-util.cc
index 789723e..5d1c0fa 100644
--- a/be/src/catalog/catalog-util.cc
+++ b/be/src/catalog/catalog-util.cc
@@ -20,9 +20,9 @@
 #include <sstream>
 
 #include "catalog/catalog-util.h"
-#include "common/status.h"
 #include "exec/read-write-util.h"
 #include "util/compress.h"
+#include "util/jni-util.h"
 #include "util/debug-util.h"
 
 #include "common/names.h"
@@ -31,6 +31,94 @@ using boost::algorithm::to_upper_copy;
 
 namespace impala {
 
+jclass JniCatalogCacheUpdateIterator::pair_cl;
+jmethodID JniCatalogCacheUpdateIterator::pair_ctor;
+jclass JniCatalogCacheUpdateIterator::boolean_cl;
+jmethodID JniCatalogCacheUpdateIterator::boolean_ctor;
+
+Status JniCatalogCacheUpdateIterator::InitJNI() {
+  JNIEnv* env = getJNIEnv();
+  if (env == nullptr) return Status("Failed to get/create JVM");
+  RETURN_IF_ERROR(
+      JniUtil::GetGlobalClassRef(env, "Lorg/apache/impala/common/Pair;", &pair_cl));
+  pair_ctor = env->GetMethodID(pair_cl, "<init>",
+      "(Ljava/lang/Object;Ljava/lang/Object;)V");
+  RETURN_ERROR_IF_EXC(env);
+  RETURN_IF_ERROR(
+      JniUtil::GetGlobalClassRef(env, "Ljava/lang/Boolean;", &boolean_cl));
+  boolean_ctor = env->GetMethodID(boolean_cl, "<init>", "(Z)V");
+  RETURN_ERROR_IF_EXC(env);
+  return Status::OK();
+}
+
+Status JniCatalogCacheUpdateIterator::createPair(JNIEnv* env, bool deleted,
+    const uint8_t* buffer, long size, jobject* out) {
+  jobject deleted_obj = env->NewObject(boolean_cl, boolean_ctor,
+      static_cast<jboolean>(deleted));
+  RETURN_ERROR_IF_EXC(env);
+  jobject byte_buffer = env->NewDirectByteBuffer(const_cast<uint8_t*>(buffer), size);
+  RETURN_ERROR_IF_EXC(env);
+  *out = env->NewObject(pair_cl, pair_ctor, deleted_obj, byte_buffer);
+  RETURN_ERROR_IF_EXC(env);
+  return Status::OK();
+}
+
+jobject TopicItemSpanIterator::next(JNIEnv* env) {
+  while (begin_ != end_) {
+    jobject result;
+    Status s;
+    const TTopicItem* current = begin_++;
+    if (decompress_) {
+      s = DecompressCatalogObject(
+          reinterpret_cast<const uint8_t*>(current->value.data()),
+          static_cast<uint32_t>(current->value.size()), &decompressed_buffer_);
+      if (!s.ok()) {
+        LOG(ERROR) << "Error decompressing catalog object: " << s.GetDetail();
+        continue;
+      }
+      s = createPair(env, current->deleted,
+          reinterpret_cast<const uint8_t*>(decompressed_buffer_.data()),
+          static_cast<long>(decompressed_buffer_.size()), &result);
+    } else {
+      s = createPair(env, current->deleted,
+          reinterpret_cast<const uint8_t*>(current->value.data()),
+          static_cast<long>(current->value.size()), &result);
+    }
+    if (s.ok()) return result;
+    LOG(ERROR) << "Error creating return value: " << s.GetDetail();
+  }
+  return nullptr;
+}
+
+jobject CatalogUpdateResultIterator::next(JNIEnv* env) {
+  const vector<TCatalogObject>& removed = result_.removed_catalog_objects;
+  const vector<TCatalogObject>& updated = result_.updated_catalog_objects;
+  while (pos_ != removed.size() + updated.size()) {
+    bool deleted;
+    const TCatalogObject* current_obj;
+    if (pos_ < removed.size()) {
+      current_obj = &removed[pos_];
+      deleted = true;
+    } else {
+      current_obj = &updated[pos_ - removed.size()];
+      deleted = false;
+    }
+    ++pos_;
+    uint8_t* buf;
+    uint32_t buf_size;
+    Status s = serializer_.Serialize(current_obj, &buf_size, &buf);
+    if (!s.ok()) {
+      LOG(ERROR) << "Error serializing catalog object: " << s.GetDetail();
+      continue;
+    }
+    jobject result = nullptr;
+    s = createPair(env, deleted, buf, buf_size, &result);
+    if (s.ok()) return result;
+    LOG(ERROR) << "Error creating jobject." << s.GetDetail();
+  }
+  return nullptr;
+}
+
 TCatalogObjectType::type TCatalogObjectTypeFromName(const string& name) {
   const string& upper = to_upper_copy(name);
   if (upper == "DATABASE") {
@@ -136,79 +224,31 @@ Status TCatalogObjectFromObjectName(const TCatalogObjectType::type& object_type,
   return Status::OK();
 }
 
-string TCatalogObjectToEntryKey(const TCatalogObject& catalog_object) {
-  // The key format is: "TCatalogObjectType:<fully qualified object name>"
-  stringstream entry_key;
-  entry_key << PrintTCatalogObjectType(catalog_object.type) << ":";
-  switch (catalog_object.type) {
-    case TCatalogObjectType::DATABASE:
-      entry_key << catalog_object.db.db_name;
-      break;
-    case TCatalogObjectType::TABLE:
-    case TCatalogObjectType::VIEW:
-      entry_key << catalog_object.table.db_name << "." << catalog_object.table.tbl_name;
-      break;
-    case TCatalogObjectType::FUNCTION:
-      entry_key << catalog_object.fn.name.db_name << "."
-                << catalog_object.fn.signature;
-      break;
-    case TCatalogObjectType::CATALOG:
-      entry_key << catalog_object.catalog.catalog_service_id;
-      break;
-    case TCatalogObjectType::DATA_SOURCE:
-      entry_key << catalog_object.data_source.name;
-      break;
-    case TCatalogObjectType::HDFS_CACHE_POOL:
-      entry_key << catalog_object.cache_pool.pool_name;
-      break;
-    case TCatalogObjectType::ROLE:
-      entry_key << catalog_object.role.role_name;
-      break;
-    case TCatalogObjectType::PRIVILEGE:
-      entry_key << catalog_object.privilege.role_id << "."
-                << catalog_object.privilege.privilege_name;
-      break;
-    default:
-      break;
-  }
-  return entry_key.str();
-}
-
-Status CompressCatalogObject(string* catalog_object) {
+Status CompressCatalogObject(const uint8_t* src, uint32_t size, string* dst) {
   scoped_ptr<Codec> compressor;
   RETURN_IF_ERROR(Codec::CreateCompressor(nullptr, false, THdfsCompression::LZ4,
       &compressor));
-  string output_buffer;
-  int64_t compressed_data_len = compressor->MaxOutputLen(catalog_object->size());
-  DCHECK_GT(compressed_data_len, 0);
+  int64_t compressed_data_len = compressor->MaxOutputLen(size);
   int64_t output_buffer_len = compressed_data_len + sizeof(uint32_t);
-  output_buffer.resize(output_buffer_len);
-  uint8_t* output_buffer_ptr =
-      const_cast<uint8_t*>(reinterpret_cast<const uint8_t*>(output_buffer.data()));
-  ReadWriteUtil::PutInt(output_buffer_ptr, static_cast<uint32_t>(catalog_object->size()));
+  dst->resize(static_cast<size_t>(output_buffer_len));
+  uint8_t* output_buffer_ptr = reinterpret_cast<uint8_t*>(&((*dst)[0]));
+  ReadWriteUtil::PutInt(output_buffer_ptr, size);
   output_buffer_ptr += sizeof(uint32_t);
-  RETURN_IF_ERROR(compressor->ProcessBlock(true, catalog_object->size(),
-      reinterpret_cast<const uint8_t*>(catalog_object->data()), &compressed_data_len,
+  RETURN_IF_ERROR(compressor->ProcessBlock(true, size, src, &compressed_data_len,
       &output_buffer_ptr));
-  output_buffer.resize(compressed_data_len + sizeof(uint32_t));
-  *catalog_object = move(output_buffer);
+  dst->resize(compressed_data_len + sizeof(uint32_t));
   return Status::OK();
 }
 
-Status DecompressCatalogObject(const string& compressed_catalog_object,
-    vector<uint8_t>* output_buffer) {
+Status DecompressCatalogObject(const uint8_t* src, uint32_t size, string* dst) {
   scoped_ptr<Codec> decompressor;
   RETURN_IF_ERROR(Codec::CreateDecompressor(nullptr, false, THdfsCompression::LZ4,
       &decompressor));
-  const uint8_t* input_data_ptr =
-      reinterpret_cast<const uint8_t*>(compressed_catalog_object.data());
-  int64_t decompressed_len = ReadWriteUtil::GetInt<uint32_t>(input_data_ptr);
-  output_buffer->resize(decompressed_len);
-  input_data_ptr += sizeof(uint32_t);
-  uint8_t* decompressed_data_ptr = output_buffer->data();
-  int64_t compressed_data_len = compressed_catalog_object.size() - sizeof(uint32_t);
-  RETURN_IF_ERROR(decompressor->ProcessBlock(true, compressed_data_len,
-      input_data_ptr, &decompressed_len, &decompressed_data_ptr));
+  int64_t decompressed_len = ReadWriteUtil::GetInt<uint32_t>(src);
+  dst->resize(static_cast<size_t>(decompressed_len));
+  uint8_t* decompressed_data_ptr = reinterpret_cast<uint8_t*>(&((*dst)[0]));
+  RETURN_IF_ERROR(decompressor->ProcessBlock(true, size - sizeof(uint32_t),
+      src + sizeof(uint32_t), &decompressed_len, &decompressed_data_ptr));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog-util.h
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-util.h b/be/src/catalog/catalog-util.h
index e98cd38..a01e9bb 100644
--- a/be/src/catalog/catalog-util.h
+++ b/be/src/catalog/catalog-util.h
@@ -19,14 +19,75 @@
 #ifndef IMPALA_CATALOG_CATALOG_UTIL_H
 #define IMPALA_CATALOG_CATALOG_UTIL_H
 
+#include <jni.h>
+#include <gen-cpp/StatestoreService_types.h>
+#include <gen-cpp/CatalogService_types.h>
+#include <rpc/thrift-util.h>
+
 #include "common/status.h"
 #include "gen-cpp/CatalogObjects_types.h"
 
 namespace impala {
 
-/// Contains utility functions for working with TCatalogObjects and their related types.
+/// A helper class used to pass catalog object updates to the FE. With this iterator, the
+/// catalog objects are decompressed and transferred to the FE one by one without having
+/// to keep the entire uncompressed catalog objects in memory.
+class JniCatalogCacheUpdateIterator {
+ public:
+  /// Initialize JNI classes and method IDs. Currently it is only initilized in impalad.
+  static Status InitJNI();
+
+  /// Return the next catalog object from a catalog update. The return type is
+  /// Pair<Boolean, ByteBuffer>. The Boolean value is true if the update is a delete
+  /// operation. The ByteBuffer is the serialized TCatalogObject. null is returned at the
+  /// end of the update set. The return value is invalided on the next call.
+  /// If the deserialization or decompression of an object is unsuccessful, the object
+  /// will be skipped and the next valid object is returned.
+  virtual jobject next(JNIEnv* env) = 0;
+
+  virtual ~JniCatalogCacheUpdateIterator() = default;
+
+ protected:
+  /// A helper function used to create the return value of next().
+  Status createPair(JNIEnv* env, bool deleted, const uint8_t* buffer, long size,
+      jobject* out);
+
+ private:
+  static jclass pair_cl;
+  static jmethodID pair_ctor;
+  static jclass boolean_cl;
+  static jmethodID boolean_ctor;
+};
+
+/// Pass catalog objects in CatalogUpdateCallback().
+class TopicItemSpanIterator : public JniCatalogCacheUpdateIterator {
+ public:
+  TopicItemSpanIterator(const vector<TTopicItem>& items, bool decompress) :
+      begin_(items.data()), end_(items.data() + items.size()),
+      decompress_(decompress) {}
 
-class Status;
+  jobject next(JNIEnv* env) override;
+
+ private:
+  const TTopicItem* begin_;
+  const TTopicItem* end_;
+  bool decompress_;
+  std::string decompressed_buffer_;
+};
+
+/// Pass catalog objects in ProcessCatalogUpdateResult().
+class CatalogUpdateResultIterator : public JniCatalogCacheUpdateIterator {
+ public:
+  explicit CatalogUpdateResultIterator(const TCatalogUpdateResult& catalog_update_result)
+      : result_(catalog_update_result), pos_(0), serializer_(false) {}
+
+  jobject next(JNIEnv* env) override;
+
+ private:
+  const TCatalogUpdateResult& result_;
+  int pos_;
+  ThriftSerializer serializer_;
+};
 
 /// Converts a string to the matching TCatalogObjectType enum type. Returns
 /// TCatalogObjectType::UNKNOWN if no match was found.
@@ -37,23 +98,17 @@ TCatalogObjectType::type TCatalogObjectTypeFromName(const std::string& name);
 Status TCatalogObjectFromObjectName(const TCatalogObjectType::type& object_type,
     const std::string& object_name, TCatalogObject* catalog_object);
 
-/// Builds and returns the topic entry key string for the given TCatalogObject. The key
-/// format is: "TCatalogObjectType:<fully qualified object name>". So a table named
-/// "foo" in a database named "bar" would have a key of: "TABLE:bar.foo"
-/// Returns an empty string if there were any problem building the key.
-std::string TCatalogObjectToEntryKey(const TCatalogObject& catalog_object);
-
-/// Compresses a serialized catalog object using LZ4 and stores it back in
-/// 'catalog_object'. Stores the size of the uncopressed catalog object in the
-/// first sizeof(uint32_t) bytes of 'catalog_object'.
-Status CompressCatalogObject(std::string* catalog_object) WARN_UNUSED_RESULT;
-
-/// Decompress an LZ4-compressed catalog object. The decompressed object
-/// is stored in 'output_buffer'. The first sizeof(uint32_t) bytes of
-/// 'compressed_catalog_object' store the size of the uncompressed catalog
-/// object.
-Status DecompressCatalogObject(const std::string& compressed_catalog_object,
-    std::vector<uint8_t>* output_buffer) WARN_UNUSED_RESULT;
+/// Compresses a serialized catalog object using LZ4 and stores it back in 'dst'. Stores
+/// the size of the uncompressed catalog object in the first sizeof(uint32_t) bytes of
+/// 'dst'. The compression fails if the uncompressed data size exceeds 0x7E000000 bytes.
+Status CompressCatalogObject(const uint8_t* src, uint32_t size, std::string* dst)
+    WARN_UNUSED_RESULT;
+
+/// Decompress an LZ4-compressed catalog object. The decompressed object is stored in
+/// 'dst'. The first sizeof(uint32_t) bytes of 'src' store the size of the uncompressed
+/// catalog object.
+Status DecompressCatalogObject(const uint8_t* src, uint32_t size, std::string* dst)
+    WARN_UNUSED_RESULT;
 
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog.cc b/be/src/catalog/catalog.cc
index d96d23e..dcc1657 100644
--- a/be/src/catalog/catalog.cc
+++ b/be/src/catalog/catalog.cc
@@ -99,8 +99,10 @@ Status Catalog::GetCatalogVersion(long* version) {
   return Status::OK();
 }
 
-Status Catalog::GetCatalogDelta(long from_version, TGetCatalogDeltaResponse* resp) {
+Status Catalog::GetCatalogDelta(CatalogServer* caller, int64_t from_version,
+    TGetCatalogDeltaResponse* resp) {
   TGetCatalogDeltaRequest request;
+  request.__set_native_catalog_server_ptr(reinterpret_cast<int64_t>(caller));
   request.__set_from_version(from_version);
   return JniUtil::CallJniMethod(catalog_, get_catalog_delta_id_, request, resp);
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/catalog/catalog.h
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog.h b/be/src/catalog/catalog.h
index 13e4529..872ceca 100644
--- a/be/src/catalog/catalog.h
+++ b/be/src/catalog/catalog.h
@@ -27,6 +27,8 @@
 
 namespace impala {
 
+class CatalogServer;
+
 /// The Catalog is a proxy for the Java-side JniCatalog class. The interface is a set of
 /// wrapper functions for methods called over JNI.
 class Catalog {
@@ -58,8 +60,11 @@ class Catalog {
 
   /// Retrieves the catalog objects that were added/modified/deleted since version
   /// 'from_version'. Returns OK if the operation was successful, otherwise a Status
-  /// object with information on the error will be returned.
-  Status GetCatalogDelta(long from_version, TGetCatalogDeltaResponse* resp);
+  /// object with information on the error will be returned. 'caller' is a pointer to
+  /// the caller CatalogServer object. caller->AddTopicUpdate() will be repeatedly
+  /// called by the frontend.
+  Status GetCatalogDelta(CatalogServer* caller, int64_t from_version,
+      TGetCatalogDeltaResponse* resp);
 
   /// Gets the Thrift representation of a Catalog object. The request is a TCatalogObject
   /// which has the desired TCatalogObjectType and name properly set.

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/rpc/thrift-util.h
----------------------------------------------------------------------
diff --git a/be/src/rpc/thrift-util.h b/be/src/rpc/thrift-util.h
index 1a66286..24b0b6f 100644
--- a/be/src/rpc/thrift-util.h
+++ b/be/src/rpc/thrift-util.h
@@ -49,12 +49,11 @@ class ThriftSerializer {
 
   /// Serializes obj into result.  Result will contain a copy of the memory.
   template <class T>
-  Status Serialize(T* obj, std::vector<uint8_t>* result) {
-    uint32_t len = 0;
-    uint8_t* buffer = NULL;
-    RETURN_IF_ERROR(Serialize<T>(obj, &len, &buffer));
-    result->resize(len);
-    memcpy(&((*result)[0]), buffer, len);
+  Status Serialize(const T* obj, std::vector<uint8_t>* result) {
+    uint32_t len;
+    uint8_t* buffer;
+    RETURN_IF_ERROR(Serialize(obj, &len, &buffer));
+    result->assign(buffer, buffer + len);
     return Status::OK();
   }
 
@@ -62,7 +61,7 @@ class ThriftSerializer {
   /// memory returned is owned by this object and will be invalid when another object
   /// is serialized.
   template <class T>
-  Status Serialize(T* obj, uint32_t* len, uint8_t** buffer) {
+  Status Serialize(const T* obj, uint32_t* len, uint8_t** buffer) {
     try {
       mem_buffer_->resetBuffer();
       obj->write(protocol_.get());
@@ -76,7 +75,7 @@ class ThriftSerializer {
   }
 
   template <class T>
-  Status Serialize(T* obj, std::string* result) {
+  Status Serialize(const T* obj, std::string* result) {
     try {
       mem_buffer_->resetBuffer();
       obj->write(protocol_.get());
@@ -94,15 +93,6 @@ class ThriftSerializer {
   boost::shared_ptr<apache::thrift::protocol::TProtocol> protocol_;
 };
 
-class ThriftDeserializer {
- public:
-  ThriftDeserializer(bool compact);
-
- private:
-  boost::shared_ptr<apache::thrift::protocol::TProtocolFactory> factory_;
-  boost::shared_ptr<apache::thrift::protocol::TProtocol> tproto_;
-};
-
 /// Utility to create a protocol (deserialization) object for 'mem'.
 boost::shared_ptr<apache::thrift::protocol::TProtocol>
 CreateDeserializeProtocol(

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index 0a84d09..12ac874 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -20,7 +20,9 @@
 #include "service/fe-support.h"
 
 #include <boost/scoped_ptr.hpp>
+#include <catalog/catalog-util.h>
 
+#include "catalog/catalog-server.h"
 #include "codegen/llvm-codegen.h"
 #include "common/init.h"
 #include "common/logging.h"
@@ -424,6 +426,65 @@ Java_org_apache_impala_service_FeSupport_NativeLookupSymbol(
   return result_bytes;
 }
 
+// Add a catalog update to pending_topic_updates_.
+extern "C"
+JNIEXPORT void JNICALL
+Java_org_apache_impala_service_FeSupport_NativeAddPendingTopicItem(JNIEnv* env,
+    jclass caller_class, jlong native_catalog_server_ptr, jstring key,
+    jbyteArray serialized_object, jboolean deleted) {
+  std::string key_string;
+  {
+    JniUtfCharGuard key_str;
+    if (!JniUtfCharGuard::create(env, key, &key_str).ok()) return;
+    key_string.assign(key_str.get());
+  }
+  JniScopedArrayCritical obj_buf;
+  if (!JniScopedArrayCritical::Create(env, serialized_object, &obj_buf)) return;
+  reinterpret_cast<CatalogServer*>(native_catalog_server_ptr)->AddPendingTopicItem(
+      std::move(key_string), obj_buf.get(), static_cast<uint32_t>(obj_buf.size()),
+      deleted);
+}
+
+// Get the next catalog update pointed by 'callback_ctx'.
+extern "C"
+JNIEXPORT jobject JNICALL
+Java_org_apache_impala_service_FeSupport_NativeGetNextCatalogObjectUpdate(JNIEnv* env,
+    jclass caller_class, jlong native_iterator_ptr) {
+  return reinterpret_cast<JniCatalogCacheUpdateIterator*>(native_iterator_ptr)->next(env);
+}
+
+extern "C"
+JNIEXPORT jboolean JNICALL
+Java_org_apache_impala_service_FeSupport_NativeLibCacheSetNeedsRefresh(JNIEnv* env,
+    jclass caller_class, jstring hdfs_location) {
+  string str;
+  {
+    JniUtfCharGuard hdfs_location_data;
+    if (!JniUtfCharGuard::create(env, hdfs_location, &hdfs_location_data).ok()) {
+      return static_cast<jboolean>(false);
+    }
+    str.assign(hdfs_location_data.get());
+  }
+  LibCache::instance()->SetNeedsRefresh(str);
+  return static_cast<jboolean>(false);
+}
+
+extern "C"
+JNIEXPORT jboolean JNICALL
+Java_org_apache_impala_service_FeSupport_NativeLibCacheRemoveEntry(JNIEnv* env,
+    jclass caller_class, jstring hdfs_lib_file) {
+  string str;
+  {
+    JniUtfCharGuard hdfs_lib_file_data;
+    if (!JniUtfCharGuard::create(env, hdfs_lib_file, &hdfs_lib_file_data).ok()) {
+      return static_cast<jboolean>(false);
+    }
+    str.assign(hdfs_lib_file_data.get());
+  }
+  LibCache::instance()->RemoveEntry(str);
+  return static_cast<jboolean>(true);
+}
+
 // Calls in to the catalog server to request prioritizing the loading of metadata for
 // specific catalog objects.
 extern "C"
@@ -478,28 +539,45 @@ namespace impala {
 
 static JNINativeMethod native_methods[] = {
   {
-    (char*)"NativeFeTestInit", (char*)"()V",
-    (void*)::Java_org_apache_impala_service_FeSupport_NativeFeTestInit
+      (char*)"NativeFeTestInit", (char*)"()V",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeFeTestInit
+  },
+  {
+      (char*)"NativeEvalExprsWithoutRow", (char*)"([B[B)[B",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeEvalExprsWithoutRow
+  },
+  {
+      (char*)"NativeCacheJar", (char*)"([B)[B",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeCacheJar
+  },
+  {
+      (char*)"NativeLookupSymbol", (char*)"([B)[B",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeLookupSymbol
+  },
+  {
+      (char*)"NativePrioritizeLoad", (char*)"([B)[B",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativePrioritizeLoad
   },
   {
-    (char*)"NativeEvalExprsWithoutRow", (char*)"([B[B)[B",
-    (void*)::Java_org_apache_impala_service_FeSupport_NativeEvalExprsWithoutRow
+      (char*)"NativeParseQueryOptions", (char*)"(Ljava/lang/String;[B)[B",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeParseQueryOptions
   },
   {
-    (char*)"NativeCacheJar", (char*)"([B)[B",
-    (void*)::Java_org_apache_impala_service_FeSupport_NativeCacheJar
+      (char*)"NativeAddPendingTopicItem", (char*)"(JLjava/lang/String;[BZ)Z",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeAddPendingTopicItem
   },
   {
-    (char*)"NativeLookupSymbol", (char*)"([B)[B",
-    (void*)::Java_org_apache_impala_service_FeSupport_NativeLookupSymbol
+      (char*)"NativeGetNextCatalogObjectUpdate",
+      (char*)"(J)Lorg/apache/impala/common/Pair;",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeGetNextCatalogObjectUpdate
   },
   {
-    (char*)"NativePrioritizeLoad", (char*)"([B)[B",
-    (void*)::Java_org_apache_impala_service_FeSupport_NativePrioritizeLoad
+      (char*)"NativeLibCacheSetNeedsRefresh", (char*)"(Ljava/lang/String;)Z",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeLibCacheSetNeedsRefresh
   },
   {
-    (char*)"NativeParseQueryOptions", (char*)"(Ljava/lang/String;[B)[B",
-    (void*)::Java_org_apache_impala_service_FeSupport_NativeParseQueryOptions
+      (char*)"NativeLibCacheRemoveEntry", (char*)"(Ljava/lang/String;)Z",
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeLibCacheRemoveEntry
   },
 };
 

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/service/frontend.cc
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index 50883af..db54158 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -69,7 +69,7 @@ Frontend::Frontend() {
     {"getHadoopConfig", "([B)[B", &get_hadoop_config_id_},
     {"getAllHadoopConfigs", "()[B", &get_hadoop_configs_id_},
     {"checkConfiguration", "()Ljava/lang/String;", &check_config_id_},
-    {"updateCatalogCache", "([[B)[B", &update_catalog_cache_id_},
+    {"updateCatalogCache", "([B)[B", &update_catalog_cache_id_},
     {"updateMembership", "([B)V", &update_membership_id_},
     {"getTableNames", "([B)[B", &get_table_names_id_},
     {"describeDb", "([B)[B", &describe_db_id_},
@@ -109,7 +109,7 @@ Frontend::Frontend() {
   ABORT_IF_ERROR(JniUtil::LocalToGlobalRef(jni_env, fe, &fe_));
 }
 
-Status Frontend::UpdateCatalogCache(const vector<TUpdateCatalogCacheRequest>& req,
+Status Frontend::UpdateCatalogCache(const TUpdateCatalogCacheRequest& req,
     TUpdateCatalogCacheResponse* resp) {
   return JniUtil::CallJniMethod(fe_, update_catalog_cache_id_, req, resp);
 }
@@ -267,4 +267,4 @@ Status Frontend::GetTableFiles(const TShowFilesParams& params, TResultSet* resul
 Status Frontend::BuildTestDescriptorTable(const TBuildTestDescriptorTableParams& params,
     TDescriptorTable* result) {
   return JniUtil::CallJniMethod(fe_, build_test_descriptor_table_id_, params, result);
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/service/frontend.h
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.h b/be/src/service/frontend.h
index 4881220..a152b7f 100644
--- a/be/src/service/frontend.h
+++ b/be/src/service/frontend.h
@@ -38,10 +38,11 @@ class Frontend {
   /// or if there is any further exception, the constructor will terminate the process.
   Frontend();
 
-  /// Request to update the Impalad catalog cache. The req argument contains a vector of
-  /// updates that each contain objects that should be added/removed from the Catalog.
-  /// Returns a response that contains details such as the new max catalog version.
-  Status UpdateCatalogCache(const vector<TUpdateCatalogCacheRequest>& req,
+  /// Request to update the Impalad catalog cache. The 'req' argument contains a pointer
+  /// to a CatalogServer used for the FE to call NativeGetNextCatalogTopicItem() back to
+  /// get the catalog objects iteratively. Returns a response that contains details such
+  /// as the new max catalog version.
+  Status UpdateCatalogCache(const TUpdateCatalogCacheRequest& req,
       TUpdateCatalogCacheResponse *resp);
 
   /// Request to update the Impalad frontend cluster membership snapshot.  The

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index af79180..0c5f75b 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -228,9 +228,6 @@ const string ImpalaServer::AUDIT_EVENT_LOG_FILE_PREFIX = "impala_audit_event_log
 const string LINEAGE_LOG_FILE_PREFIX = "impala_lineage_log_1.0-";
 
 const uint32_t MAX_CANCELLATION_QUEUE_SIZE = 65536;
-// Max size for multiple update in a single split. JNI is not able to write java byte
-// array more than 2GB. A single topic update is not restricted by this.
-const uint64_t MAX_CATALOG_UPDATE_BATCH_SIZE_BYTES = 500 * 1024 * 1024;
 
 const string BEESWAX_SERVER_NAME = "beeswax-frontend";
 const string HS2_SERVER_NAME = "hiveserver2-frontend";
@@ -1327,144 +1324,39 @@ void ImpalaServer::CatalogUpdateCallback(
       incoming_topic_deltas.find(CatalogServer::IMPALA_CATALOG_TOPIC);
   if (topic == incoming_topic_deltas.end()) return;
   const TTopicDelta& delta = topic->second;
-
-  // Update catalog cache in frontend. An update is split into batches of size
-  // MAX_CATALOG_UPDATE_BATCH_SIZE_BYTES each for multiple updates. IMPALA-3499
-  if (delta.topic_entries.size() != 0)  {
-    vector<TCatalogObject> dropped_objects;
-    vector<TUpdateCatalogCacheRequest> update_reqs;
-    update_reqs.push_back(TUpdateCatalogCacheRequest());
-    TUpdateCatalogCacheRequest* incremental_request = &update_reqs.back();
-    incremental_request->__set_is_delta(delta.is_delta);
-    // Process all Catalog updates (new and modified objects) and determine what the
-    // new catalog version will be.
-    int64_t new_catalog_version = catalog_update_info_.catalog_version;
-    uint64_t batch_size_bytes = 0;
-    for (const TTopicItem& item: delta.topic_entries) {
-      Status status;
-      vector<uint8_t> data_buffer;
-      const uint8_t* data_buffer_ptr = nullptr;
-      uint32_t len = 0;
-      if (FLAGS_compact_catalog_topic) {
-        status = DecompressCatalogObject(item.value, &data_buffer);
-        if (!status.ok()) {
-          LOG(ERROR) << "Error decompressing catalog object " << item.key
-                     << ": " << status.GetDetail();
-          continue;
-        }
-        data_buffer_ptr = data_buffer.data();
-        len = data_buffer.size();
-      } else {
-        data_buffer_ptr = reinterpret_cast<const uint8_t*>(item.value.data());
-        len = item.value.size();
-      }
-      if (len > 100 * 1024 * 1024 /* 100MB */) {
-        LOG(INFO) << "Received large catalog object(>100mb): "
-            << item.key << " is "
-            << PrettyPrinter::Print(len, TUnit::BYTES);
-      }
-      TCatalogObject catalog_object;
-      status = DeserializeThriftMsg(data_buffer_ptr, &len, FLAGS_compact_catalog_topic,
-          &catalog_object);
-      if (!status.ok()) {
-        LOG(ERROR) << "Error deserializing item " << item.key
-            << ": " << status.GetDetail();
-        continue;
-      }
-
-      if (batch_size_bytes + len > MAX_CATALOG_UPDATE_BATCH_SIZE_BYTES) {
-        // Initialize a new batch of catalog updates.
-        update_reqs.push_back(TUpdateCatalogCacheRequest());
-        incremental_request = &update_reqs.back();
-        batch_size_bytes = 0;
-      }
-
-      if (catalog_object.type == TCatalogObjectType::CATALOG) {
-        incremental_request->__set_catalog_service_id(
-            catalog_object.catalog.catalog_service_id);
-        new_catalog_version = catalog_object.catalog_version;
-      }
-      VLOG(3) << (item.deleted ? "Deleted " : "Added ") << "item: " << item.key
-          << " version: " << catalog_object.catalog_version << " of size: " << len;
-
-      if (!item.deleted) {
-        // Refresh the lib cache entries of any added functions and data sources
-        // TODO: if frontend returns the list of functions and data sources, we do not
-        // need to deserialize these in backend.
-        if (catalog_object.type == TCatalogObjectType::FUNCTION) {
-          DCHECK(catalog_object.__isset.fn);
-          LibCache::instance()->SetNeedsRefresh(catalog_object.fn.hdfs_location);
-        }
-        if (catalog_object.type == TCatalogObjectType::DATA_SOURCE) {
-          DCHECK(catalog_object.__isset.data_source);
-          LibCache::instance()->SetNeedsRefresh(catalog_object.data_source.hdfs_location);
-        }
-        incremental_request->updated_objects.push_back(catalog_object);
-      } else {
-        // We need to look up any dropped functions and data sources and remove
-        // them from the library cache.
-        if (catalog_object.type == TCatalogObjectType::FUNCTION ||
-            catalog_object.type == TCatalogObjectType::DATA_SOURCE) {
-          TCatalogObject existing_object;
-          if (exec_env_->frontend()->GetCatalogObject(
-              catalog_object, &existing_object).ok()) {
-            // If the object exists in the catalog it may have been dropped and
-            // re-created. To avoid removing the re-created object's entry from
-            // the cache verify that the existing object's version <= the
-            // version of the dropped object included in this statestore
-            // heartbeat.
-            DCHECK_NE(existing_object.catalog_version, catalog_object.catalog_version);
-            if (existing_object.catalog_version < catalog_object.catalog_version) {
-              dropped_objects.push_back(existing_object);
-            }
-          }
-        }
-        incremental_request->removed_objects.push_back(catalog_object);
-      }
-      batch_size_bytes += len;
-    }
-
-    // Call the FE to apply the changes to the Impalad Catalog.
-    TUpdateCatalogCacheResponse resp;
-    Status s = exec_env_->frontend()->UpdateCatalogCache(update_reqs, &resp);
-    if (!s.ok()) {
-      LOG(ERROR) << "There was an error processing the impalad catalog update. Requesting"
-                 << " a full topic update to recover: " << s.GetDetail();
-      subscriber_topic_updates->push_back(TTopicDelta());
-      TTopicDelta& update = subscriber_topic_updates->back();
-      update.topic_name = CatalogServer::IMPALA_CATALOG_TOPIC;
-      update.__set_from_version(0L);
-      ImpaladMetrics::CATALOG_READY->SetValue(false);
-      // Dropped all cached lib files (this behaves as if all functions and data
-      // sources are dropped).
-      LibCache::instance()->DropCache();
-    } else {
-      {
-        unique_lock<mutex> unique_lock(catalog_version_lock_);
-        catalog_update_info_.catalog_version = new_catalog_version;
-        catalog_update_info_.catalog_topic_version = delta.to_version;
-        catalog_update_info_.catalog_service_id = resp.catalog_service_id;
-        catalog_update_info_.min_catalog_object_version = resp.min_catalog_object_version;
-        LOG(INFO) << "Catalog topic update applied with version: " << new_catalog_version
-            << " new min catalog object version: " << resp.min_catalog_object_version;
-      }
-      ImpaladMetrics::CATALOG_READY->SetValue(new_catalog_version > 0);
-      // TODO: deal with an error status
-      discard_result(UpdateCatalogMetrics());
-      // Remove all dropped objects from the library cache.
-      // TODO: is this expensive? We'd like to process heartbeats promptly.
-      for (TCatalogObject& object: dropped_objects) {
-        if (object.type == TCatalogObjectType::FUNCTION) {
-          LibCache::instance()->RemoveEntry(object.fn.hdfs_location);
-        } else if (object.type == TCatalogObjectType::DATA_SOURCE) {
-          LibCache::instance()->RemoveEntry(object.data_source.hdfs_location);
-        } else {
-          DCHECK(false);
-        }
-      }
+  TopicItemSpanIterator callback_ctx (delta.topic_entries, FLAGS_compact_catalog_topic);
+
+  TUpdateCatalogCacheRequest req;
+  req.__set_is_delta(delta.is_delta);
+  req.__set_native_iterator_ptr(reinterpret_cast<int64_t>(&callback_ctx));
+  TUpdateCatalogCacheResponse resp;
+  Status s = exec_env_->frontend()->UpdateCatalogCache(req, &resp);
+  if (!s.ok()) {
+    LOG(ERROR) << "There was an error processing the impalad catalog update. Requesting"
+               << " a full topic update to recover: " << s.GetDetail();
+    subscriber_topic_updates->emplace_back();
+    TTopicDelta& update = subscriber_topic_updates->back();
+    update.topic_name = CatalogServer::IMPALA_CATALOG_TOPIC;
+    update.__set_from_version(0L);
+    ImpaladMetrics::CATALOG_READY->SetValue(false);
+    // Dropped all cached lib files (this behaves as if all functions and data
+    // sources are dropped).
+    LibCache::instance()->DropCache();
+  } else {
+    {
+      unique_lock<mutex> unique_lock(catalog_version_lock_);
+      catalog_update_info_.catalog_version = resp.new_catalog_version;
+      catalog_update_info_.catalog_topic_version = delta.to_version;
+      catalog_update_info_.catalog_service_id = resp.catalog_service_id;
+      catalog_update_info_.min_catalog_object_version = resp.min_catalog_object_version;
+      LOG(INFO) << "Catalog topic update applied with version: " <<
+          resp.new_catalog_version << " new min catalog object version: " <<
+          resp.min_catalog_object_version;
     }
+    ImpaladMetrics::CATALOG_READY->SetValue(resp.new_catalog_version > 0);
+    // TODO: deal with an error status
+    discard_result(UpdateCatalogMetrics());
   }
-
   // Always update the minimum subscriber version for the catalog topic.
   {
     unique_lock<mutex> unique_lock(catalog_version_lock_);
@@ -1555,20 +1447,17 @@ Status ImpalaServer::ProcessCatalogUpdateResult(
       WaitForCatalogUpdateTopicPropagation(catalog_service_id);
     }
   } else {
-    // Operation with a result set.
+    CatalogUpdateResultIterator callback_ctx(catalog_update_result);
     TUpdateCatalogCacheRequest update_req;
     update_req.__set_is_delta(true);
+    update_req.__set_native_iterator_ptr(reinterpret_cast<int64_t>(&callback_ctx));
+    // The catalog version is updated in WaitForCatalogUpdate below. So we need a
+    // standalone field in the request to update the service ID without touching the
+    // catalog version.
     update_req.__set_catalog_service_id(catalog_update_result.catalog_service_id);
-    if (catalog_update_result.__isset.updated_catalog_objects) {
-      update_req.__set_updated_objects(catalog_update_result.updated_catalog_objects);
-    }
-    if (catalog_update_result.__isset.removed_catalog_objects) {
-      update_req.__set_removed_objects(catalog_update_result.removed_catalog_objects);
-    }
     // Apply the changes to the local catalog cache.
     TUpdateCatalogCacheResponse resp;
-    Status status = exec_env_->frontend()->UpdateCatalogCache(
-        vector<TUpdateCatalogCacheRequest>{update_req}, &resp);
+    Status status = exec_env_->frontend()->UpdateCatalogCache(update_req, &resp);
     if (!status.ok()) LOG(ERROR) << status.GetDetail();
     RETURN_IF_ERROR(status);
     if (!wait_for_all_subscribers) return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/service/impalad-main.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc
index a7f6abd..e0049d0 100644
--- a/be/src/service/impalad-main.cc
+++ b/be/src/service/impalad-main.cc
@@ -22,6 +22,7 @@
 #include <unistd.h>
 #include <jni.h>
 
+#include "catalog/catalog-util.h"
 #include "common/logging.h"
 #include "common/init.h"
 #include "exec/hbase-table-scanner.h"
@@ -66,6 +67,7 @@ int ImpaladMain(int argc, char** argv) {
   ABORT_IF_ERROR(HBaseTable::InitJNI());
   ABORT_IF_ERROR(HBaseTableWriter::InitJNI());
   ABORT_IF_ERROR(HiveUdfCall::InitEnv());
+  ABORT_IF_ERROR(JniCatalogCacheUpdateIterator::InitJNI());
   InitFeSupport();
 
   if (FLAGS_enable_rm) {

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/util/jni-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.cc b/be/src/util/jni-util.cc
index 3128697..db7893b 100644
--- a/be/src/util/jni-util.cc
+++ b/be/src/util/jni-util.cc
@@ -30,7 +30,8 @@ namespace impala {
 Status JniUtfCharGuard::create(JNIEnv* env, jstring jstr, JniUtfCharGuard* out) {
   DCHECK(jstr != nullptr);
   DCHECK(!env->ExceptionCheck());
-  const char* utf_chars = env->GetStringUTFChars(jstr, nullptr);
+  jboolean is_copy;
+  const char* utf_chars = env->GetStringUTFChars(jstr, &is_copy);
   bool exception_check = static_cast<bool>(env->ExceptionCheck());
   if (utf_chars == nullptr || exception_check) {
     if (exception_check) env->ExceptionClear();
@@ -45,6 +46,24 @@ Status JniUtfCharGuard::create(JNIEnv* env, jstring jstr, JniUtfCharGuard* out)
   return Status::OK();
 }
 
+bool JniScopedArrayCritical::Create(JNIEnv* env, jbyteArray jarr,
+    JniScopedArrayCritical* out) {
+  DCHECK(env != nullptr);
+  DCHECK(out != nullptr);
+  DCHECK(!env->ExceptionCheck());
+  int size = env->GetArrayLength(jarr);
+  void* pac = env->GetPrimitiveArrayCritical(jarr, nullptr);
+  if (pac == nullptr) {
+    LOG(ERROR) << "GetPrimitiveArrayCritical() failed. Probable OOM on JVM side";
+    return false;
+  }
+  out->env_ = env;
+  out->jarr_ = jarr;
+  out->arr_ = static_cast<uint8_t*>(pac);
+  out->size_ = size;
+  return true;
+}
+
 jclass JniUtil::jni_util_cl_ = NULL;
 jclass JniUtil::internal_exc_cl_ = NULL;
 jmethodID JniUtil::get_jvm_metrics_id_ = NULL;

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/be/src/util/jni-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h
index 43ac131..f0afb66 100644
--- a/be/src/util/jni-util.h
+++ b/be/src/util/jni-util.h
@@ -168,6 +168,36 @@ class JniUtfCharGuard {
   DISALLOW_COPY_AND_ASSIGN(JniUtfCharGuard);
 };
 
+class JniScopedArrayCritical {
+ public:
+  /// Construct a JniScopedArrayCritical holding nothing.
+  JniScopedArrayCritical():  env_(nullptr), jarr_(nullptr), arr_(nullptr), size_(0) {}
+
+  /// Release the held byte[] contents if necessary.
+  ~JniScopedArrayCritical() {
+    if (env_ != nullptr && jarr_ != nullptr && arr_ != nullptr) {
+      env_->ReleasePrimitiveArrayCritical(jarr_, arr_, JNI_ABORT);
+    }
+  }
+
+  /// Try to get the contents of 'jarr' via JNIEnv::GetPrimitiveArrayCritical() and set
+  /// the results in 'out'. Returns true upon success and false otherwise. If false is
+  /// returned 'out' is not modified.
+  static bool Create(JNIEnv* env, jbyteArray jarr, JniScopedArrayCritical* out)
+      WARN_UNUSED_RESULT;
+
+  uint8_t* get() const { return arr_; }
+
+  int size() const { return size_; }
+ private:
+  JNIEnv* env_;
+  jbyteArray jarr_;
+  uint8_t* arr_;
+  int size_;
+  DISALLOW_COPY_AND_ASSIGN(JniScopedArrayCritical);
+};
+
+
 /// Utility class for JNI-related functionality.
 /// Init() should be called as soon as the native library is loaded.
 /// Creates global class references, and promotes local references to global references.
@@ -316,30 +346,6 @@ class JniUtil {
     return Status::OK();
   }
 
-  template <typename T, typename R>
-  static Status CallJniMethod(const jobject& obj, const jmethodID& method,
-      const vector<T>& args, R* response) {
-    JNIEnv* jni_env = getJNIEnv();
-    JniLocalFrame jni_frame;
-    RETURN_IF_ERROR(jni_frame.push(jni_env));
-    jclass jByteArray_class = jni_env->FindClass("[B");
-    jobjectArray array_of_jByteArray =
-        jni_env->NewObjectArray(args.size(), jByteArray_class, NULL);
-    RETURN_ERROR_IF_EXC(jni_env);
-    jbyteArray request_bytes;
-    for (int i = 0; i < args.size(); i++) {
-      RETURN_IF_ERROR(SerializeThriftMsg(jni_env, &args[i], &request_bytes));
-      jni_env->SetObjectArrayElement(array_of_jByteArray, i, request_bytes);
-      RETURN_ERROR_IF_EXC(jni_env);
-      jni_env->DeleteLocalRef(request_bytes);
-    }
-    jbyteArray result_bytes = static_cast<jbyteArray>(
-        jni_env->CallObjectMethod(obj, method, array_of_jByteArray));
-    RETURN_ERROR_IF_EXC(jni_env);
-    RETURN_IF_ERROR(DeserializeThriftMsg(jni_env, result_bytes, response));
-    return Status::OK();
-  }
-
   template <typename T>
   static Status CallJniMethod(const jobject& obj, const jmethodID& method,
       const T& arg, std::string* response) {

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/common/thrift/CatalogInternalService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/CatalogInternalService.thrift b/common/thrift/CatalogInternalService.thrift
index ffccd4b..6007f02 100644
--- a/common/thrift/CatalogInternalService.thrift
+++ b/common/thrift/CatalogInternalService.thrift
@@ -26,21 +26,23 @@ include "CatalogObjects.thrift"
 struct TGetCatalogDeltaRequest {
   // The base catalog version from which the delta is computed.
   1: required i64 from_version
+
+  // The native caller ptr for calling back NativeAddPendingTopicItem().
+  2: required i64 native_catalog_server_ptr
 }
 
-// Response from a call to GetCatalogDelta. Contains a delta of catalog objects
-// (databases, tables/views, and functions) from the CatalogService's cache relative (>)
-// to the catalog version specified in TGetCatalogDelta.from_version.
+// Response from a call to GetCatalogDelta. The catalog object updates are passed
+// separately via NativeAddPendingTopicItem() callback.
 struct TGetCatalogDeltaResponse {
   // The maximum catalog version of all objects in this response or 0 if the Catalog
   // contained no objects.
   1: required i64 max_catalog_version
 
   // List of updated (new and modified) catalog objects whose catalog verion is
-  // larger than TGetCatalotDeltaRequest.from_version.
-  2: required list<CatalogObjects.TCatalogObject> updated_objects
+  // larger than TGetCatalotDeltaRequest.from_version. Deprecated after IMPALA-5990.
+  2: optional list<CatalogObjects.TCatalogObject> updated_objects_deprecated
 
   // List of deleted catalog objects whose catalog version is larger than
-  // TGetCatalogDelta.from_version.
-  3: required list<CatalogObjects.TCatalogObject> deleted_objects
+  // TGetCatalogDelta.from_version. Deprecated after IMPALA-5990.
+  3: optional list<CatalogObjects.TCatalogObject> deleted_objects_deprecated
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/common/thrift/Frontend.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index ba21605..b81c1a1 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -655,20 +655,25 @@ struct TSymbolLookupResult {
 }
 
 // Sent from the impalad BE to FE with the results of each CatalogUpdate heartbeat.
-// Contains details on all catalog objects that need to be updated.
+// The catalog object updates are passed separately via NativeGetCatalogUpdate() callback.
 struct TUpdateCatalogCacheRequest {
   // True if update only contains entries changed from the previous update. Otherwise,
   // contains the entire topic.
   1: required bool is_delta
 
-  // The Catalog Service ID this update came from.
-  2: required Types.TUniqueId catalog_service_id
+  // The Catalog Service ID this update came from. A request should has either this field
+  // set or a Catalog typed catalog object in the update list.
+  2: optional Types.TUniqueId catalog_service_id
 
-  // New or modified items. Empty list if no items were updated.
-  3: required list<CatalogObjects.TCatalogObject> updated_objects
+  // New or modified items. Empty list if no items were updated. Deprecated after
+  // IMPALA-5990.
+  3: optional list<CatalogObjects.TCatalogObject> updated_objects_deprecated
 
-  // Empty if no items were removed or is_delta is false.
-  4: required list<CatalogObjects.TCatalogObject> removed_objects
+  // Empty if no items were removed or is_delta is false. Deprecated after IMPALA-5990.
+  4: optional list<CatalogObjects.TCatalogObject> removed_objects_deprecated
+
+  // The native ptr for calling back NativeGetCatalogUpdate().
+  5: required i64 native_iterator_ptr
 }
 
 // Response from a TUpdateCatalogCacheRequest.
@@ -678,6 +683,9 @@ struct TUpdateCatalogCacheResponse {
 
   // The minimum catalog object version after CatalogUpdate() was processed.
   2: required i64 min_catalog_object_version
+
+  // The updated catalog version needed by the backend.
+  3: required i64 new_catalog_version
 }
 
 // Sent from the impalad BE to FE with the latest cluster membership snapshot resulting

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/catalog/Catalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Catalog.java b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
index 6136835..ff7b1e4 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
@@ -567,6 +567,8 @@ public abstract class Catalog {
             catalogObject.getCache_pool().getPool_name().toLowerCase();
       case DATA_SOURCE:
         return "DATA_SOURCE:" + catalogObject.getData_source().getName().toLowerCase();
+      case CATALOG:
+        return "CATALOG:" + catalogObject.getCatalog().catalog_service_id;
       default:
         throw new IllegalStateException(
             "Unsupported catalog object type: " + catalogObject.getType());

http://git-wip-us.apache.org/repos/asf/impala/blob/a2431638/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index 7bc2a91..a3b0cb0 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -23,6 +23,7 @@ import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -55,12 +56,12 @@ import org.apache.impala.common.JniUtil;
 import org.apache.impala.common.Pair;
 import org.apache.impala.common.Reference;
 import org.apache.impala.hive.executor.UdfExecutor;
+import org.apache.impala.service.FeSupport;
 import org.apache.impala.thrift.TCatalog;
 import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TCatalogUpdateResult;
 import org.apache.impala.thrift.TFunction;
-import org.apache.impala.thrift.TGetCatalogDeltaResponse;
 import org.apache.impala.thrift.TGetCatalogUsageResponse;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TPrivilege;
@@ -72,6 +73,8 @@ import org.apache.impala.util.PatternMatcher;
 import org.apache.impala.util.SentryProxy;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;
 
 import com.codahale.metrics.Timer;
@@ -387,54 +390,85 @@ public class CatalogServiceCatalog extends Catalog {
   }
 
   /**
-   * Identifies and returns the catalog objects that were added/modified/deleted in the
-   * catalog with versions > 'fromVersion'. It operates on a snaphsot of the catalog
-   * without holding the catalog lock which means that other concurrent metadata
-   * operations can still make progress while the catalog delta is computed. An entry in
-   * the topic update log is added for every catalog object that is included in the
-   * catalog delta. The log is examined by operations using SYNC_DDL to determine which
-   * topic update covers the result set of metadata operation. Once the catalog delta is
-   * computed, the entries in the delete log with versions less than 'fromVersion' are
-   * garbage collected.
+   * The context for add*ToCatalogDelta(), called by getCatalogDelta. It contains
+   * callback information, version range and collected topics.
    */
-  public TGetCatalogDeltaResponse getCatalogDelta(long fromVersion) {
-    // Maximum catalog version (inclusive) to be included in the catalog delta.
-    long toVersion = getCatalogVersion();
-    TGetCatalogDeltaResponse resp = new TGetCatalogDeltaResponse();
-    resp.setUpdated_objects(new ArrayList<TCatalogObject>());
-    resp.setDeleted_objects(new ArrayList<TCatalogObject>());
-    resp.setMax_catalog_version(toVersion);
+  class GetCatalogDeltaContext {
+    // The CatalogServer pointer for NativeAddPendingTopicItem() callback.
+    long nativeCatalogServerPtr;
+    // The from and to version of this delta.
+    long fromVersion;
+    long toVersion;
+    // The keys of the updated topics.
+    Set<String> updatedCatalogObjects;
+    TSerializer serializer;
+
+    GetCatalogDeltaContext(long nativeCatalogServerPtr, long fromVersion, long toVersion)
+    {
+      this.nativeCatalogServerPtr = nativeCatalogServerPtr;
+      this.fromVersion = fromVersion;
+      this.toVersion = toVersion;
+      updatedCatalogObjects = new HashSet<>();
+      serializer = new TSerializer(new TBinaryProtocol.Factory());
+    }
+
+    void addCatalogObject(TCatalogObject obj, boolean delete) throws TException {
+      String key = Catalog.toCatalogObjectKey(obj);
+      if (obj.type != TCatalogObjectType.CATALOG) {
+        topicUpdateLog_.add(key,
+            new TopicUpdateLog.Entry(0, obj.getCatalog_version(), toVersion));
+        if (!delete) updatedCatalogObjects.add(key);
+      }
+      // TODO: TSerializer.serialize() returns a copy of the internal byte array, which
+      // could be elided.
+      byte[] data = serializer.serialize(obj);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Collected catalog " + (delete ? "deletion: " : "update: ") + key +
+            " version: " + obj.catalog_version);
+      }
+      if (!FeSupport.NativeAddPendingTopicItem(nativeCatalogServerPtr, key, data, delete))
+      {
+        LOG.error("NativeAddPendingTopicItem failed in BE. key=" + key + ", delete="
+            + delete + ", data_size=" + data.length);
+      }
+    }
+  }
 
+  /**
+   * Identifies the catalog objects that were added/modified/deleted in the catalog with
+   * versions > 'fromVersion'. It operates on a snaphsot of the catalog without holding
+   * the catalog lock which means that other concurrent metadata operations can still make
+   * progress while the catalog delta is computed. An entry in the topic update log is
+   * added for every catalog object that is included in the catalog delta. The log is
+   * examined by operations using SYNC_DDL to determine which topic update covers the
+   * result set of metadata operation. Once the catalog delta is computed, the entries in
+   * the delete log with versions less than 'fromVersion' are garbage collected.
+   * The catalog delta is passed to the backend by calling NativeAddPendingTopicItem().
+   */
+  public long getCatalogDelta(long nativeCatalogServerPtr, long fromVersion) throws
+      TException {
+    GetCatalogDeltaContext ctx = new GetCatalogDeltaContext(nativeCatalogServerPtr,
+        fromVersion, getCatalogVersion());
     for (Db db: getAllDbs()) {
-      addDatabaseToCatalogDelta(db, fromVersion, toVersion, resp);
+      addDatabaseToCatalogDelta(db, ctx);
     }
     for (DataSource dataSource: getAllDataSources()) {
-      addDataSourceToCatalogDelta(dataSource, fromVersion, toVersion, resp);
+      addDataSourceToCatalogDelta(dataSource, ctx);
     }
     for (HdfsCachePool cachePool: getAllHdfsCachePools()) {
-      addHdfsCachePoolToCatalogDelta(cachePool, fromVersion, toVersion, resp);
+      addHdfsCachePoolToCatalogDelta(cachePool, ctx);
     }
     for (Role role: getAllRoles()) {
-      addRoleToCatalogDelta(role, fromVersion, toVersion, resp);
+      addRoleToCatalogDelta(role, ctx);
     }
-    Set<String> updatedCatalogObjects = Sets.newHashSet();
-    for (TCatalogObject catalogObj: resp.updated_objects) {
-      topicUpdateLog_.add(Catalog.toCatalogObjectKey(catalogObj),
-          new TopicUpdateLog.Entry(0, catalogObj.getCatalog_version(),
-              toVersion));
-      updatedCatalogObjects.add(Catalog.toCatalogObjectKey(catalogObj));
-    }
-
     // Identify the catalog objects that were removed from the catalog for which their
-    // versions are in range ('fromVersion', 'toVersion']. We need to make sure
+    // versions are in range ('ctx.fromVersion', 'ctx.toVersion']. We need to make sure
     // that we don't include "deleted" objects that were re-added to the catalog.
-    for (TCatalogObject removedObject: getDeletedObjects(fromVersion, toVersion)) {
-      if (!updatedCatalogObjects.contains(
+    for (TCatalogObject removedObject:
+        getDeletedObjects(ctx.fromVersion, ctx.toVersion)) {
+      if (!ctx.updatedCatalogObjects.contains(
           Catalog.toCatalogObjectKey(removedObject))) {
-        topicUpdateLog_.add(Catalog.toCatalogObjectKey(removedObject),
-            new TopicUpdateLog.Entry(0, removedObject.getCatalog_version(),
-                toVersion));
-        resp.addToDeleted_objects(removedObject);
+        ctx.addCatalogObject(removedObject, true);
       }
     }
     // Each topic update should contain a single "TCatalog" object which is used to
@@ -443,20 +477,21 @@ public class CatalogServiceCatalog extends Catalog {
     // version at this point, it ensures impalads will always bump their versions,
     // even in the case where an object has been dropped.
     TCatalogObject catalog =
-        new TCatalogObject(TCatalogObjectType.CATALOG, toVersion);
+        new TCatalogObject(TCatalogObjectType.CATALOG, ctx.toVersion);
     catalog.setCatalog(new TCatalog(catalogServiceId_));
-    resp.addToUpdated_objects(catalog);
+    ctx.addCatalogObject(catalog, false);
     // Garbage collect the delete and topic update log.
-    deleteLog_.garbageCollect(toVersion);
-    topicUpdateLog_.garbageCollectUpdateLogEntries(toVersion);
-    lastSentTopicUpdate_.set(toVersion);
+    deleteLog_.garbageCollect(ctx.toVersion);
+    topicUpdateLog_.garbageCollectUpdateLogEntries(ctx.toVersion);
+    lastSentTopicUpdate_.set(ctx.toVersion);
     // Notify any operation that is waiting on the next topic update.
     synchronized (topicUpdateLog_) {
       topicUpdateLog_.notifyAll();
     }
-    return resp;
+    return ctx.toVersion;
   }
 
+
   /**
    * Get a snapshot view of all the catalog objects that were deleted between versions
    * ('fromVersion', 'toVersion'].
@@ -520,23 +555,23 @@ public class CatalogServiceCatalog extends Catalog {
 
   /**
    * Adds a database in the topic update if its version is in the range
-   * ('fromVersion', 'toVersion']. It iterates through all the tables and functions of
-   * this database to determine if they can be included in the topic update.
+   * ('ctx.fromVersion', 'ctx.toVersion']. It iterates through all the tables and
+   * functions of this database to determine if they can be included in the topic update.
    */
-  private void addDatabaseToCatalogDelta(Db db, long fromVersion, long toVersion,
-      TGetCatalogDeltaResponse resp) {
+  private void addDatabaseToCatalogDelta(Db db, GetCatalogDeltaContext ctx)
+      throws TException {
     long dbVersion = db.getCatalogVersion();
-    if (dbVersion > fromVersion && dbVersion <= toVersion) {
+    if (dbVersion > ctx.fromVersion && dbVersion <= ctx.toVersion) {
       TCatalogObject catalogDb =
           new TCatalogObject(TCatalogObjectType.DATABASE, dbVersion);
       catalogDb.setDb(db.toThrift());
-      resp.addToUpdated_objects(catalogDb);
+      ctx.addCatalogObject(catalogDb, false);
     }
     for (Table tbl: getAllTables(db)) {
-      addTableToCatalogDelta(tbl, fromVersion, toVersion, resp);
+      addTableToCatalogDelta(tbl, ctx);
     }
     for (Function fn: getAllFunctions(db)) {
-      addFunctionToCatalogDelta(fn, fromVersion, toVersion, resp);
+      addFunctionToCatalogDelta(fn, ctx);
     }
   }
 
@@ -568,25 +603,25 @@ public class CatalogServiceCatalog extends Catalog {
 
   /**
    * Adds a table in the topic update if its version is in the range
-   * ('fromVersion', 'toVersion']. If the table's version is larger than 'toVersion' and
-   * the table has skipped a topic update 'MAX_NUM_SKIPPED_TOPIC_UPDATES' times, it is
-   * included in the topic update. This prevents tables that are updated frequently from
-   * skipping topic updates indefinitely, which would also violate the semantics of
-   * SYNC_DDL.
+   * ('ctx.fromVersion', 'ctx.toVersion']. If the table's version is larger than
+   * 'ctx.toVersion' and the table has skipped a topic update
+   * 'MAX_NUM_SKIPPED_TOPIC_UPDATES' times, it is included in the topic update. This
+   * prevents tables that are updated frequently from skipping topic updates indefinitely,
+   * which would also violate the semantics of SYNC_DDL.
    */
-  private void addTableToCatalogDelta(Table tbl, long fromVersion, long toVersion,
-      TGetCatalogDeltaResponse resp) {
-    if (tbl.getCatalogVersion() <= toVersion) {
-      addTableToCatalogDeltaHelper(tbl, fromVersion, toVersion, resp);
+  private void addTableToCatalogDelta(Table tbl, GetCatalogDeltaContext ctx)
+      throws TException  {
+    if (tbl.getCatalogVersion() <= ctx.toVersion) {
+      addTableToCatalogDeltaHelper(tbl, ctx);
     } else {
       TopicUpdateLog.Entry topicUpdateEntry =
           topicUpdateLog_.getOrCreateLogEntry(tbl.getUniqueName());
       Preconditions.checkNotNull(topicUpdateEntry);
-      if (topicUpdateEntry.getNumSkippedTopicUpdates() >= MAX_NUM_SKIPPED_TOPIC_UPDATES) {
-        addTableToCatalogDeltaHelper(tbl, fromVersion, toVersion, resp);
+      if (topicUpdateEntry.getNumSkippedTopicUpdates() == MAX_NUM_SKIPPED_TOPIC_UPDATES) {
+        addTableToCatalogDeltaHelper(tbl, ctx);
       } else {
         LOG.info("Table " + tbl.getFullName() + " is skipping topic update " +
-            toVersion);
+            ctx.toVersion);
         topicUpdateLog_.add(tbl.getUniqueName(),
             new TopicUpdateLog.Entry(
                 topicUpdateEntry.getNumSkippedTopicUpdates() + 1,
@@ -598,23 +633,23 @@ public class CatalogServiceCatalog extends Catalog {
 
   /**
    * Helper function that tries to add a table in a topic update. It acquires table's
-   * lock and checks if its version is in the ('fromVersion', 'toVersion'] range and how
-   * many consecutive times (if any) has the table skipped a topic update.
+   * lock and checks if its version is in the ('ctx.fromVersion', 'ctx.toVersion'] range
+   * and how many consecutive times (if any) has the table skipped a topic update.
    */
-  private void addTableToCatalogDeltaHelper(Table tbl, long fromVersion, long toVersion,
-      TGetCatalogDeltaResponse resp) {
+  private void addTableToCatalogDeltaHelper(Table tbl, GetCatalogDeltaContext ctx)
+      throws TException {
     TCatalogObject catalogTbl =
         new TCatalogObject(TCatalogObjectType.TABLE, Catalog.INITIAL_CATALOG_VERSION);
     tbl.getLock().lock();
     try {
       long tblVersion = tbl.getCatalogVersion();
-      if (tblVersion <= fromVersion) return;
+      if (tblVersion <= ctx.fromVersion) return;
       TopicUpdateLog.Entry topicUpdateEntry =
           topicUpdateLog_.getOrCreateLogEntry(tbl.getUniqueName());
-      if (tblVersion > toVersion &&
+      if (tblVersion > ctx.toVersion &&
           topicUpdateEntry.getNumSkippedTopicUpdates() < MAX_NUM_SKIPPED_TOPIC_UPDATES) {
         LOG.info("Table " + tbl.getFullName() + " is skipping topic update " +
-            toVersion);
+            ctx.toVersion);
         topicUpdateLog_.add(tbl.getUniqueName(),
             new TopicUpdateLog.Entry(
                 topicUpdateEntry.getNumSkippedTopicUpdates() + 1,
@@ -630,7 +665,7 @@ public class CatalogServiceCatalog extends Catalog {
         return;
       }
       catalogTbl.setCatalog_version(tbl.getCatalogVersion());
-      resp.addToUpdated_objects(catalogTbl);
+      ctx.addCatalogObject(catalogTbl, false);
     } finally {
       tbl.getLock().unlock();
     }
@@ -638,65 +673,65 @@ public class CatalogServiceCatalog extends Catalog {
 
   /**
    * Adds a function to the topic update if its version is in the range
-   * ('fromVersion', 'toVersion'].
+   * ('ctx.fromVersion', 'ctx.toVersion'].
    */
-  private void addFunctionToCatalogDelta(Function fn, long fromVersion, long toVersion,
-      TGetCatalogDeltaResponse resp) {
+  private void addFunctionToCatalogDelta(Function fn, GetCatalogDeltaContext ctx)
+      throws TException {
     long fnVersion = fn.getCatalogVersion();
-    if (fnVersion <= fromVersion || fnVersion > toVersion) return;
+    if (fnVersion <= ctx.fromVersion || fnVersion > ctx.toVersion) return;
     TCatalogObject function =
         new TCatalogObject(TCatalogObjectType.FUNCTION, fnVersion);
     function.setFn(fn.toThrift());
-    resp.addToUpdated_objects(function);
+    ctx.addCatalogObject(function, false);
   }
 
   /**
    * Adds a data source to the topic update if its version is in the range
-   * ('fromVersion', 'toVersion'].
+   * ('ctx.fromVersion', 'ctx.toVersion'].
    */
-  private void addDataSourceToCatalogDelta(DataSource dataSource, long fromVersion,
-      long toVersion, TGetCatalogDeltaResponse resp) {
+  private void addDataSourceToCatalogDelta(DataSource dataSource,
+      GetCatalogDeltaContext ctx) throws TException  {
     long dsVersion = dataSource.getCatalogVersion();
-    if (dsVersion <= fromVersion || dsVersion > toVersion) return;
+    if (dsVersion <= ctx.fromVersion || dsVersion > ctx.toVersion) return;
     TCatalogObject catalogObj =
         new TCatalogObject(TCatalogObjectType.DATA_SOURCE, dsVersion);
     catalogObj.setData_source(dataSource.toThrift());
-    resp.addToUpdated_objects(catalogObj);
+    ctx.addCatalogObject(catalogObj, false);
   }
 
   /**
    * Adds a HDFS cache pool to the topic update if its version is in the range
-   * ('fromVersion', 'toVersion'].
+   * ('ctx.fromVersion', 'ctx.toVersion'].
    */
-  private void addHdfsCachePoolToCatalogDelta(HdfsCachePool cachePool, long fromVersion,
-      long toVersion, TGetCatalogDeltaResponse resp) {
+  private void addHdfsCachePoolToCatalogDelta(HdfsCachePool cachePool,
+      GetCatalogDeltaContext ctx) throws TException  {
     long cpVersion = cachePool.getCatalogVersion();
-    if (cpVersion <= fromVersion || cpVersion > toVersion) {
+    if (cpVersion <= ctx.fromVersion || cpVersion > ctx.toVersion) {
       return;
     }
     TCatalogObject pool =
         new TCatalogObject(TCatalogObjectType.HDFS_CACHE_POOL, cpVersion);
     pool.setCache_pool(cachePool.toThrift());
-    resp.addToUpdated_objects(pool);
+    ctx.addCatalogObject(pool, false);
   }
 
 
   /**
    * Adds a role to the topic update if its version is in the range
-   * ('fromVersion', 'toVersion']. It iterates through all the privileges of this role to
-   * determine if they can be inserted in the topic update.
+   * ('ctx.fromVersion', 'ctx.toVersion']. It iterates through all the privileges of
+   * this role to determine if they can be inserted in the topic update.
    */
-  private void addRoleToCatalogDelta(Role role, long fromVersion, long toVersion,
-      TGetCatalogDeltaResponse resp) {
+  private void addRoleToCatalogDelta(Role role, GetCatalogDeltaContext ctx)
+      throws TException  {
     long roleVersion = role.getCatalogVersion();
-    if (roleVersion > fromVersion && roleVersion <= toVersion) {
+    if (roleVersion > ctx.fromVersion && roleVersion <= ctx.toVersion) {
       TCatalogObject thriftRole =
           new TCatalogObject(TCatalogObjectType.ROLE, roleVersion);
       thriftRole.setRole(role.toThrift());
-      resp.addToUpdated_objects(thriftRole);
+      ctx.addCatalogObject(thriftRole, false);
     }
     for (RolePrivilege p: getAllPrivileges(role)) {
-      addRolePrivilegeToCatalogDelta(p, fromVersion, toVersion, resp);
+      addRolePrivilegeToCatalogDelta(p, ctx);
     }
   }
 
@@ -715,16 +750,16 @@ public class CatalogServiceCatalog extends Catalog {
 
   /**
    * Adds a role privilege to the topic update if its version is in the range
-   * ('fromVersion', 'toVersion'].
+   * ('ctx.fromVersion', 'ctx.toVersion'].
    */
-  private void addRolePrivilegeToCatalogDelta(RolePrivilege priv, long fromVersion,
-      long toVersion, TGetCatalogDeltaResponse resp) {
+  private void addRolePrivilegeToCatalogDelta(RolePrivilege priv,
+      GetCatalogDeltaContext ctx) throws TException  {
     long privVersion = priv.getCatalogVersion();
-    if (privVersion <= fromVersion || privVersion > toVersion) return;
+    if (privVersion <= ctx.fromVersion || privVersion > ctx.toVersion) return;
     TCatalogObject privilege =
         new TCatalogObject(TCatalogObjectType.PRIVILEGE, privVersion);
     privilege.setPrivilege(priv.toThrift());
-    resp.addToUpdated_objects(privilege);
+    ctx.addCatalogObject(privilege, false);
   }
 
   /**


[15/21] impala git commit: Revert 'IMPALA-6338: Fix flaky test_profile_fragment_instances'

Posted by ta...@apache.org.
Revert 'IMPALA-6338: Fix flaky test_profile_fragment_instances'

There have been several crashes observed in testing recently, and its
not clear what's going on, so for now revert this moderately risky
change.

Change-Id: I48c11f0817c5190a3a94f8260f3e8ef653357ab3
Reviewed-on: http://gerrit.cloudera.org:8080/9243
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Thomas Tauber-Marshall <tm...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 1ddb15621a2b028a00a1a0a489d2a49224f17a43
Parents: 9e887b0
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Wed Feb 7 11:12:47 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/common/status.h                      |  6 ------
 be/src/runtime/coordinator-backend-state.cc | 12 ++++++------
 be/src/runtime/coordinator-backend-state.h  |  6 ++----
 be/src/runtime/coordinator.cc               | 10 ++++------
 be/src/runtime/coordinator.h                |  4 +---
 tests/query_test/test_observability.py      | 10 +++++-----
 6 files changed, 18 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/1ddb1562/be/src/common/status.h
----------------------------------------------------------------------
diff --git a/be/src/common/status.h b/be/src/common/status.h
index f0f91f7..24dba8b 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -95,13 +95,7 @@ class NODISCARD Status {
   static Status MemLimitExceeded();
   static Status MemLimitExceeded(const std::string& details);
 
-  /// Indicates a 'cancelled' status. CANCELLED should not be reported by a fragment
-  /// instance that encounters a problem - instances should return a specific error,
-  /// and then the coordinator will initiate cancellation.
-  /// TODO: we use this in some places to indicate things other than query cancellation,
-  /// which can be confusing.
   static const Status CANCELLED;
-
   static const Status DEPRECATED_RPC;
 
   /// Copy c'tor makes copy of error detail so Status can be returned by value.

http://git-wip-us.apache.org/repos/asf/impala/blob/1ddb1562/be/src/runtime/coordinator-backend-state.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index b238cad..914a3e4 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -234,7 +234,7 @@ void Coordinator::BackendState::LogFirstInProgress(
 }
 
 inline bool Coordinator::BackendState::IsDone() const {
-  return num_remaining_instances_ == 0 || (!status_.ok() && !status_.IsCancelled());
+  return num_remaining_instances_ == 0 || !status_.ok();
 }
 
 bool Coordinator::BackendState::ApplyExecStatusReport(
@@ -338,8 +338,8 @@ bool Coordinator::BackendState::Cancel() {
   // Nothing to cancel if the exec rpc was not sent
   if (!rpc_sent_) return false;
 
-  // don't cancel if it already finished (for any reason) or cancelled
-  if (IsDone() || status_.IsCancelled()) return false;
+  // don't cancel if it already finished (for any reason)
+  if (IsDone()) return false;
 
   /// If the status is not OK, we still try to cancel - !OK status might mean
   /// communication failure between backend and coordinator, but fragment
@@ -391,10 +391,10 @@ bool Coordinator::BackendState::Cancel() {
 void Coordinator::BackendState::PublishFilter(const TPublishFilterParams& rpc_params) {
   DCHECK_EQ(rpc_params.dst_query_id, query_id_);
   {
-    // If the backend is already done or cancelled, it's not waiting for this filter, so
-    // we skip sending it in this case.
+    // If the backend is already done, it's not waiting for this filter, so we skip
+    // sending it in this case.
     lock_guard<mutex> l(lock_);
-    if (IsDone() || status_.IsCancelled()) return;
+    if (IsDone()) return;
   }
 
   if (fragments_.count(rpc_params.dst_fragment_idx) == 0) return;

http://git-wip-us.apache.org/repos/asf/impala/blob/1ddb1562/be/src/runtime/coordinator-backend-state.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator-backend-state.h b/be/src/runtime/coordinator-backend-state.h
index 860b968..0973ca3 100644
--- a/be/src/runtime/coordinator-backend-state.h
+++ b/be/src/runtime/coordinator-backend-state.h
@@ -219,7 +219,7 @@ class Coordinator::BackendState {
 
   /// If the status indicates an error status, execution has either been aborted by the
   /// executing impalad (which then reported the error) or cancellation has been
-  /// initiated by the coordinator.
+  /// initiated; either way, execution must not be cancelled.
   Status status_;
 
   /// Used to distinguish between errors reported by a specific fragment instance,
@@ -254,9 +254,7 @@ class Coordinator::BackendState {
       const FilterRoutingTable& filter_routing_table,
       TExecQueryFInstancesParams* rpc_params);
 
-  /// Return true if execution at this backend is done. The backend is considered done if
-  /// either all instances have completed, or an error (other than cancel) is encountered.
-  /// Caller must hold lock_.
+  /// Return true if execution at this backend is done. Caller must hold lock_.
   bool IsDone() const;
 };
 

http://git-wip-us.apache.org/repos/asf/impala/blob/1ddb1562/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 05ecf9f..7973775 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -867,9 +867,6 @@ Status Coordinator::GetNext(QueryResultSet* results, int max_rows, bool* eos) {
     ReleaseExecResources();
     // wait for all backends to complete before computing the summary
     // TODO: relocate this so GetNext() won't have to wait for backends to complete?
-    // Note that doing this here allows us to ensure that a query that completes
-    // successfully will have a full runtime profile by the time that Fetch() indicates
-    // all of the results have been returned.
     RETURN_IF_ERROR(WaitForBackendCompletion());
     // Release admission control resources after backends are finished.
     ReleaseAdmissionControlResources();
@@ -923,8 +920,10 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
         Substitute("Unknown backend index $0 (max known: $1)",
             params.coord_state_idx, backend_states_.size() - 1));
   }
-  // If the query was cancelled, don't process the update.
-  if (query_status_.IsCancelled()) return Status::OK();
+  BackendState* backend_state = backend_states_[params.coord_state_idx];
+  // TODO: return here if returned_all_results_?
+  // TODO: return CANCELLED in that case? Although that makes the cancellation propagation
+  // path more irregular.
 
   // TODO: only do this when the sink is done; probably missing a done field
   // in TReportExecStatus for that
@@ -932,7 +931,6 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
     UpdateInsertExecStatus(params.insert_exec_status);
   }
 
-  BackendState* backend_state = backend_states_[params.coord_state_idx];
   if (backend_state->ApplyExecStatusReport(params, &exec_summary_, &progress_)) {
     // This report made this backend done, so update the status and
     // num_remaining_backends_.

http://git-wip-us.apache.org/repos/asf/impala/blob/1ddb1562/be/src/runtime/coordinator.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index fbbdfa9..d630b9a 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -290,9 +290,7 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
   boost::mutex lock_;
 
   /// Overall status of the entire query; set to the first reported fragment error
-  /// status or to CANCELLED, if Cancel() is called. Note that some fragments may have
-  /// status CANCELLED even if this is not CANCELLED if cancellation is initiated because
-  /// returned_all_results_ is true or an error is encountered.
+  /// status or to CANCELLED, if Cancel() is called.
   Status query_status_;
 
   /// If true, the query is done returning all results.  It is possible that the

http://git-wip-us.apache.org/repos/asf/impala/blob/1ddb1562/tests/query_test/test_observability.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_observability.py b/tests/query_test/test_observability.py
index a7508a4..85fc4f1 100644
--- a/tests/query_test/test_observability.py
+++ b/tests/query_test/test_observability.py
@@ -124,14 +124,14 @@ class TestObservability(ImpalaTestSuite):
         join (select * from l LIMIT 2000000) b on a.l_orderkey = -b.l_orderkey;""")
     # There are 3 scan nodes and each appears in the profile 4 times (for 3 fragment
     # instances + the averaged fragment).
-    assert results.runtime_profile.count("HDFS_SCAN_NODE") == 12, results.runtime_profile
+    assert results.runtime_profile.count("HDFS_SCAN_NODE") == 12
     # There are 3 exchange nodes and each appears in the profile 2 times (for 1 fragment
     # instance + the averaged fragment).
-    assert results.runtime_profile.count("EXCHANGE_NODE") == 6, results.runtime_profile
+    assert results.runtime_profile.count("EXCHANGE_NODE") == 6
     # The following appear only in the root fragment which has 1 instance.
-    assert results.runtime_profile.count("HASH_JOIN_NODE") == 2, results.runtime_profile
-    assert results.runtime_profile.count("AGGREGATION_NODE") == 2, results.runtime_profile
-    assert results.runtime_profile.count("PLAN_ROOT_SINK") == 2, results.runtime_profile
+    assert results.runtime_profile.count("HASH_JOIN_NODE") == 2
+    assert results.runtime_profile.count("AGGREGATION_NODE") == 2
+    assert results.runtime_profile.count("PLAN_ROOT_SINK") == 2
 
   # IMPALA-6399: Run this test serially to avoid a delay over the wait time in fetching
   # the profile.