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

drill git commit: DRILL-4794: Fix a premature exit of the outer loop during pruning.

Repository: drill
Updated Branches:
  refs/heads/master 34ca63ba1 -> 43776706b


DRILL-4794: Fix a premature exit of the outer loop during pruning.

close apache/drill#550


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

Branch: refs/heads/master
Commit: 43776706b161c9ff715fc7f5a9f8af76f5b2354c
Parents: 34ca63b
Author: Aman Sinha <as...@maprtech.com>
Authored: Wed Jul 20 22:36:02 2016 -0700
Committer: Aman Sinha <as...@maprtech.com>
Committed: Thu Jul 21 10:46:18 2016 -0700

----------------------------------------------------------------------
 .../logical/partition/PruneScanRule.java        |  1 -
 .../store/parquet/TestParquetMetadataCache.java | 20 ++++++++++++++++++++
 2 files changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/43776706/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
index 209e03d..82e3bb7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
@@ -308,7 +308,6 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule {
                   maxIndex = tmpIndex;
                 } else if (maxIndex != tmpIndex) {
                   isSinglePartition = false;
-                  break;
                 } else {
                   // we only want to compare until the maxIndex inclusive since subsequent values would be null
                   for (int j = 0; j <= maxIndex; j++) {

http://git-wip-us.apache.org/repos/asf/drill/blob/43776706/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
index dae8694..a5c0730 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
@@ -305,6 +305,26 @@ public class TestParquetMetadataCache extends PlanTestBase {
         new String[] {});
   }
 
+  @Test // DRILL-4794
+  public void testDrill4794() throws Exception {
+    test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName1));
+    checkForMetadataFile(tableName1);
+    String query = String.format("select dir0, dir1, o_custkey, o_orderdate from dfs_test.`%s/%s` " +
+            " where dir0=1994 or dir1='Q3'",
+        getDfsTestTmpSchemaLocation(), tableName1);
+
+    int expectedRowCount = 60;
+    int expectedNumFiles = 6;
+
+    int actualRowCount = testSql(query);
+    assertEquals(expectedRowCount, actualRowCount);
+    String numFilesPattern = "numFiles=" + expectedNumFiles;
+    String usedMetaPattern = "usedMetadataFile=true";
+    String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), tableName1);
+    PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
+        new String[] {});
+  }
+
   private void checkForMetadataFile(String table) throws Exception {
     String tmpDir = getDfsTestTmpSchemaLocation();
     String metaFile = Joiner.on("/").join(tmpDir, table, Metadata.METADATA_FILENAME);