From d7939b753f831baf15b0c4e15fa13f4fdfa6a7a1 Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Fri, 25 Mar 2016 12:55:59 -0700 Subject: [PATCH 1/7] DRILL-4530: Optimize partition pruning with metadata caching for the single partition case. - Enhance PruneScanRule to detect single partitions based on referenced dirs in the filter. - Keep a new status of EXPANDED_PARTIAL for FileSelection. - Create separate .directories metadata file to prune directories first before files. - Introduce cacheFileRoot attribute to keep track of the parent directory of the cache file after partition pruning. --- .../planner/sql/HivePartitionDescriptor.java | 3 +- .../physical/base/AbstractFileGroupScan.java | 2 +- .../exec/physical/base/FileGroupScan.java | 3 +- .../planner/AbstractPartitionDescriptor.java | 5 + .../planner/DFSFilePartitionLocation.java | 7 +- .../FileSystemPartitionDescriptor.java | 49 ++++++--- .../planner/ParquetPartitionDescriptor.java | 11 +- .../exec/planner/PartitionDescriptor.java | 7 +- .../drill/exec/planner/PartitionLocation.java | 1 + .../partition/FindPartitionConditions.java | 12 +++ .../logical/partition/PruneScanRule.java | 102 ++++++++++++++++-- .../drill/exec/store/dfs/DrillFileSystem.java | 1 - .../drill/exec/store/dfs/FileSelection.java | 93 ++++++++++++++-- .../exec/store/dfs/easy/EasyGroupScan.java | 2 +- .../drill/exec/store/parquet/Metadata.java | 94 +++++++++++++++- .../store/parquet/ParquetFormatPlugin.java | 17 ++- .../exec/store/parquet/ParquetGroupScan.java | 52 ++++++--- .../parquet/TestParquetMetadataCache.java | 24 +++-- 18 files changed, 405 insertions(+), 80 deletions(-) diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java index c8e45ca2c37..f0b68c1a95d 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java @@ -86,6 +86,7 @@ public int getMaxHierarchyLevel() { return numPartitionLevels; } + @Override public String getBaseTableLocation() { HiveReadEntry origEntry = ((HiveScan) scanRel.getGroupScan()).hiveReadEntry; return origEntry.table.getTable().getSd().getLocation(); @@ -151,7 +152,7 @@ protected void createPartitionSublists() { } @Override - public TableScan createTableScan(List newPartitions) throws Exception { + public TableScan createTableScan(List newPartitions, String cacheFileRoot) throws Exception { GroupScan newGroupScan = createNewGroupScan(newPartitions); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java index 606aa4da77b..89ceecfa469 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java @@ -38,7 +38,7 @@ public void modifyFileSelection(FileSelection selection) { } @Override - public FileGroupScan clone(FileSelection selection) throws IOException { + public FileGroupScan clone(FileSelection selection, String cacheFileRoot) throws IOException { throw new UnsupportedOperationException(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java index 552d1e8bba2..66e8375a257 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java @@ -25,5 +25,6 @@ public interface FileGroupScan extends GroupScan { public void modifyFileSelection(FileSelection selection); - public FileGroupScan clone(FileSelection selection) throws IOException; + public FileGroupScan clone(FileSelection selection, String cacheFileRoot) throws IOException; + } \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java index c9ca4488a00..66b26d8ee9c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java @@ -55,4 +55,9 @@ public Iterator> iterator() { return locationSuperList.iterator(); } + @Override + public boolean supportsSinglePartOptimization() { + return false; + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java index 6e42f3b33da..cac5d93eb30 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSFilePartitionLocation.java @@ -26,7 +26,7 @@ public class DFSFilePartitionLocation extends SimplePartitionLocation { private final String[] dirs; private final String file; - public DFSFilePartitionLocation(int max, String selectionRoot, String file) { + public DFSFilePartitionLocation(int max, String selectionRoot, String file, boolean hasDirsOnly) { this.file = file; this.dirs = new String[max]; @@ -42,8 +42,8 @@ public DFSFilePartitionLocation(int max, String selectionRoot, String file) { postPath = postPath.substring(1); } String[] mostDirs = postPath.split("/"); - int maxLoop = Math.min(max, mostDirs.length - 1); - for(int i =0; i < maxLoop; i++){ + int maxLoop = Math.min(max, hasDirsOnly ? mostDirs.length : mostDirs.length - 1); + for(int i =0; i < maxLoop; i++) { this.dirs[i] = mostDirs[i]; } } @@ -71,5 +71,6 @@ public String getEntirePartitionLocation() { public String[] getDirs() { return dirs; } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java index cfc85423dc4..6265e359bea 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java @@ -35,6 +35,7 @@ import org.apache.calcite.prepare.RelOptTableImpl; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.util.BitSets; +import org.apache.calcite.util.Pair; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos; import org.apache.drill.common.types.Types; @@ -139,15 +140,17 @@ public String getName(int index) { return partitionLabel + index; } - private String getBaseTableLocation() { + @Override + public String getBaseTableLocation() { final FormatSelection origSelection = (FormatSelection) table.getSelection(); return origSelection.getSelection().selectionRoot; } @Override protected void createPartitionSublists() { - final Collection fileLocations = getFileLocations(); + final Pair, Boolean> fileLocationsAndStatus = getFileLocationsAndStatus(); List locations = new LinkedList<>(); + boolean hasDirsOnly = fileLocationsAndStatus.right; final String selectionRoot = getBaseTableLocation(); @@ -159,8 +162,8 @@ protected void createPartitionSublists() { // Figure out the list of leaf subdirectories. For each leaf subdirectory, find the list of files (DFSFilePartitionLocation) // it contains. - for (String file: fileLocations) { - DFSFilePartitionLocation dfsFilePartitionLocation = new DFSFilePartitionLocation(MAX_NESTED_SUBDIRS, selectionRoot, file); + for (String file: fileLocationsAndStatus.left) { + DFSFilePartitionLocation dfsFilePartitionLocation = new DFSFilePartitionLocation(MAX_NESTED_SUBDIRS, selectionRoot, file, hasDirsOnly); final String[] dirs = dfsFilePartitionLocation.getDirs(); final List dirList = Arrays.asList(dirs); @@ -180,25 +183,33 @@ protected void createPartitionSublists() { sublistsCreated = true; } - protected Collection getFileLocations() { + protected Pair, Boolean> getFileLocationsAndStatus() { Collection fileLocations = null; + Pair, Boolean> fileLocationsAndStatus = null; + boolean isExpandedPartial = false; if (scanRel instanceof DrillScanRel) { // If a particular GroupScan provides files, get the list of files from there rather than // DrillTable because GroupScan would have the updated version of the selection final DrillScanRel drillScan = (DrillScanRel) scanRel; if (drillScan.getGroupScan().hasFiles()) { fileLocations = drillScan.getGroupScan().getFiles(); + isExpandedPartial = false; } else { - fileLocations = ((FormatSelection) table.getSelection()).getAsFiles(); + FileSelection selection = ((FormatSelection) table.getSelection()).getSelection(); + fileLocations = selection.getFiles(); + isExpandedPartial = selection.isExpandedPartial(); } } else if (scanRel instanceof EnumerableTableScan) { - fileLocations = ((FormatSelection) table.getSelection()).getAsFiles(); + FileSelection selection = ((FormatSelection) table.getSelection()).getSelection(); + fileLocations = selection.getFiles(); + isExpandedPartial = selection.isExpandedPartial(); } - return fileLocations; + fileLocationsAndStatus = Pair.of(fileLocations, isExpandedPartial); + return fileLocationsAndStatus; } @Override - public TableScan createTableScan(List newPartitionLocation) throws Exception { + public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot) throws Exception { List newFiles = Lists.newArrayList(); for (final PartitionLocation location : newPartitionLocation) { if (!location.isCompositePartition()) { @@ -212,8 +223,12 @@ public TableScan createTableScan(List newPartitionLocation) t } if (scanRel instanceof DrillScanRel) { - final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation()); - final FileGroupScan newGroupScan = ((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection); +// final FormatSelection formatSelection = (FormatSelection)((DynamicDrillTable)scanRel.getTable()).getSelection(); + final FormatSelection formatSelection = (FormatSelection)table.getSelection(); + final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(), + cacheFileRoot, formatSelection.getSelection().getDirStatus()); + final FileGroupScan newGroupScan = + ((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection, cacheFileRoot); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), scanRel.getTable(), @@ -222,16 +237,17 @@ public TableScan createTableScan(List newPartitionLocation) t ((DrillScanRel) scanRel).getColumns(), true /*filter pushdown*/); } else if (scanRel instanceof EnumerableTableScan) { - return createNewTableScanFromSelection((EnumerableTableScan)scanRel, newFiles); + return createNewTableScanFromSelection((EnumerableTableScan)scanRel, newFiles, cacheFileRoot); } else { throw new UnsupportedOperationException("Only DrillScanRel and EnumerableTableScan is allowed!"); } } - private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, List newFiles) { + private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, List newFiles, String cacheFileRoot) { final RelOptTableImpl t = (RelOptTableImpl) oldScan.getTable(); final FormatSelection formatSelection = (FormatSelection) table.getSelection(); - final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation()); + final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(), + cacheFileRoot, formatSelection.getSelection().getDirStatus()); final FormatSelection newFormatSelection = new FormatSelection(formatSelection.getFormat(), newFileSelection); final DrillTranslatableTable newTable = new DrillTranslatableTable( new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(), @@ -242,4 +258,9 @@ private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, L return EnumerableTableScan.create(oldScan.getCluster(), newOptTableImpl); } + @Override + public boolean supportsSinglePartOptimization() { + return true; + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java index 07e1412110a..ccc5e156604 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java @@ -80,9 +80,9 @@ public int getMaxHierarchyLevel() { return partitionColumns.size(); } - private GroupScan createNewGroupScan(List newFiles) throws IOException { + private GroupScan createNewGroupScan(List newFiles, String cacheFileRoot) throws IOException { final FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation()); - final FileGroupScan newScan = ((FileGroupScan)scanRel.getGroupScan()).clone(newSelection); + final FileGroupScan newScan = ((FileGroupScan)scanRel.getGroupScan()).clone(newSelection, cacheFileRoot); return newScan; } @@ -113,7 +113,8 @@ public TypeProtos.MajorType getVectorType(SchemaPath column, PlannerSettings pla return ((ParquetGroupScan) scanRel.getGroupScan()).getTypeForColumn(column); } - private String getBaseTableLocation() { + @Override + public String getBaseTableLocation() { final FormatSelection origSelection = (FormatSelection) scanRel.getDrillTable().getSelection(); return origSelection.getSelection().selectionRoot; } @@ -130,13 +131,13 @@ protected void createPartitionSublists() { } @Override - public TableScan createTableScan(List newPartitionLocation) throws Exception { + public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot) throws Exception { List newFiles = Lists.newArrayList(); for (final PartitionLocation location : newPartitionLocation) { newFiles.add(location.getEntirePartitionLocation()); } - final GroupScan newGroupScan = createNewGroupScan(newFiles); + final GroupScan newGroupScan = createNewGroupScan(newFiles, cacheFileRoot); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java index f08d71303a1..a9410b56ae8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java @@ -77,9 +77,14 @@ void populatePartitionVectors(ValueVector[] vectors, List par /** * Methods create a new TableScan rel node, given the lists of new partitions or new files to SCAN. * @param newPartitions + * @param cacheFileRoot - optional (can be null) * @return * @throws Exception */ - public TableScan createTableScan(List newPartitions) throws Exception; + public TableScan createTableScan(List newPartitions, String cacheFileRoot) throws Exception; + + public boolean supportsSinglePartOptimization(); + + public String getBaseTableLocation(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java index f94e8cb9660..31c926975c7 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java @@ -54,4 +54,5 @@ public interface PartitionLocation { * Returns if this is a simple or composite partition. */ public boolean isCompositePartition(); + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java index d1446b6c710..620b6b266ba 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java @@ -84,6 +84,10 @@ public void clear() { private final BitSet dirs; + // The Scan could be projecting several dirN columns but we are only interested in the + // ones that are referenced by the Filter, so keep track of such referenced dirN columns. + private final BitSet referencedDirs; + private final List pushStatusStack = Lists.newArrayList(); private final Deque opStack = new ArrayDeque(); @@ -103,6 +107,7 @@ public FindPartitionConditions(BitSet dirs) { // go deep super(true); this.dirs = dirs; + this.referencedDirs = new BitSet(dirs.size()); } public FindPartitionConditions(BitSet dirs, RexBuilder builder) { @@ -110,6 +115,7 @@ public FindPartitionConditions(BitSet dirs, RexBuilder builder) { super(true); this.dirs = dirs; this.builder = builder; + this.referencedDirs = new BitSet(dirs.size()); } public void analyze(RexNode exp) { @@ -131,6 +137,10 @@ public RexNode getFinalCondition() { return resultCondition; } + public BitSet getReferencedDirs() { + return referencedDirs; + } + private Void pushVariable() { pushStatusStack.add(PushDirFilter.NO_PUSH); return null; @@ -222,6 +232,8 @@ public Void visitInputRef(RexInputRef inputRef) { if(dirs.get(inputRef.getIndex())){ pushStatusStack.add(PushDirFilter.PUSH); addResult(inputRef); + referencedDirs.set(inputRef.getIndex()); + }else{ pushStatusStack.add(PushDirFilter.NO_PUSH); } 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 a9fb1018821..31be896cbe1 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 @@ -60,7 +60,6 @@ import org.apache.drill.exec.record.VectorContainer; import org.apache.drill.exec.store.StoragePluginOptimizerRule; import org.apache.drill.exec.store.dfs.FormatSelection; -import org.apache.drill.exec.store.parquet.ParquetGroupScan; import org.apache.drill.exec.vector.NullableBitVector; import org.apache.calcite.rel.RelNode; import org.apache.calcite.plan.RelOptRule; @@ -68,6 +67,7 @@ import org.apache.calcite.plan.RelOptRuleOperand; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rex.RexNode; +import org.apache.commons.lang3.tuple.Pair; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -79,6 +79,7 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PruneScanRule.class); final OptimizerRulesContext optimizerContext; + boolean wasAllPartitionsPruned = false; // whether all partitions were previously eliminated public PruneScanRule(RelOptRuleOperand operand, String id, OptimizerRulesContext optimizerContext) { super(operand, id); @@ -143,6 +144,11 @@ public static final RelOptRule getDirFilterOnScan(OptimizerRulesContext optimize } protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectRel, TableScan scanRel) { + if (wasAllPartitionsPruned) { + // if previously we had already pruned out all the partitions, we should exit early + return; + } + final String pruningClassName = getClass().getName(); logger.info("Beginning partition pruning, pruning class: {}", pruningClassName); Stopwatch totalPruningTime = Stopwatch.createStarted(); @@ -166,6 +172,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR List fieldNames = scanRel.getRowType().getFieldNames(); BitSet columnBitset = new BitSet(); BitSet partitionColumnBitSet = new BitSet(); + Map partitionMap = Maps.newHashMap(); int relColIndex = 0; for (String field : fieldNames) { @@ -174,6 +181,8 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR fieldNameMap.put(partitionIndex, field); partitionColumnBitSet.set(partitionIndex); columnBitset.set(relColIndex); + // mapping between the relColIndex and partitionIndex + partitionMap.put(relColIndex, partitionIndex); } relColIndex++; } @@ -193,6 +202,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR FindPartitionConditions c = new FindPartitionConditions(columnBitset, filterRel.getCluster().getRexBuilder()); c.analyze(condition); RexNode pruneCondition = c.getFinalCondition(); + BitSet referencedDirsBitSet = c.getReferencedDirs(); logger.info("Total elapsed time to build and analyze filter tree: {} ms", miscTimer.elapsed(TimeUnit.MILLISECONDS)); @@ -210,6 +220,10 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR int batchIndex = 0; PartitionLocation firstLocation = null; LogicalExpression materializedExpr = null; + boolean checkForSingle = descriptor.supportsSinglePartOptimization(); + boolean isSinglePartition = true; + String[] spInfo = null; + int maxIndex = -1; // Outer loop: iterate over a list of batches of PartitionLocations for (List partitions : descriptor) { @@ -269,13 +283,54 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR int recordCount = 0; int qualifiedCount = 0; - // Inner loop: within each batch iterate over the PartitionLocations - for(PartitionLocation part: partitions){ - if(!output.getAccessor().isNull(recordCount) && output.getAccessor().get(recordCount) == 1){ - newPartitions.add(part); - qualifiedCount++; + if (checkForSingle && + partitions.get(0).isCompositePartition() /* apply single partition check only for composite partitions */) { + // Inner loop: within each batch iterate over the PartitionLocations + for (PartitionLocation part : partitions) { + assert part.isCompositePartition(); + if(!output.getAccessor().isNull(recordCount) && output.getAccessor().get(recordCount) == 1) { + newPartitions.add(part); + if (isSinglePartition) { // only need to do this if we are already single partition + // compose the array of partition values for the directories that are referenced by filter: + // e.g suppose the dir hierarchy is year/quarter/month and the query is: + // SELECT * FROM T WHERE dir0=2015 AND dir1 = 'Q1', + // then for 2015/Q1/Feb, this will have ['2015', 'Q1', null] + // Note that we are not using the PartitionLocation here but composing a different list because + // we are only interested in the directory columns that are referenced in the filter condition. not + // the SELECT list or other parts of the query. + Pair p = composePartition(referencedDirsBitSet, partitionMap, vectors, recordCount); + String[] parts = p.getLeft(); + int tmpIndex = p.getRight(); + if (spInfo == null) { + spInfo = parts; + 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++) { + if (spInfo[j] == null // prefixes should be non-null + || !spInfo[j].equals(parts[j])) { + isSinglePartition = false; + break; + } + } + } + } + qualifiedCount++; + } + recordCount++; + } + } else { + // Inner loop: within each batch iterate over the PartitionLocations + for(PartitionLocation part: partitions){ + if(!output.getAccessor().isNull(recordCount) && output.getAccessor().get(recordCount) == 1) { + newPartitions.add(part); + qualifiedCount++; + } + recordCount++; } - recordCount++; } logger.debug("Within batch {}: total records: {}, qualified records: {}", batchIndex, recordCount, qualifiedCount); batchIndex++; @@ -306,6 +361,8 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR // In such case, we should not drop filter. newPartitions.add(firstLocation.getPartitionLocationRecursive().get(0)); canDropFilter = false; + wasAllPartitionsPruned = true; + logger.info("All {} partitions were pruned; added back a single partition to allow creating a schema", numTotal); } logger.info("Pruned {} partitions down to {}", numTotal, newPartitions.size()); @@ -320,7 +377,17 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR condition = condition.accept(reverseVisitor); pruneCondition = pruneCondition.accept(reverseVisitor); - RelNode inputRel = descriptor.createTableScan(newPartitions); + String cacheFileRoot = null; + if (checkForSingle && isSinglePartition) { + // if metadata cache file could potentially be used, then assign a proper cacheFileRoot + String path = ""; + for (int j = 0; j <= maxIndex; j++) { + path += "/" + spInfo[j]; + } + cacheFileRoot = descriptor.getBaseTableLocation() + path; + } + + RelNode inputRel = descriptor.createTableScan(newPartitions, cacheFileRoot); if (projectRel != null) { inputRel = projectRel.copy(projectRel.getTraitSet(), Collections.singletonList(inputRel)); @@ -340,6 +407,25 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR } } + private Pair composePartition(BitSet referencedDirsBitSet, + Map partitionMap, + ValueVector[] vectors, + int recordCount) { + String[] partition = new String[vectors.length]; + int maxIndex = -1; + for (int referencedDirsIndex : BitSets.toIter(referencedDirsBitSet)) { + int partitionColumnIndex = partitionMap.get(referencedDirsIndex); + ValueVector vv = vectors[partitionColumnIndex]; + if (vv.getAccessor().getValueCount() > 0 && + vv.getAccessor().getObject(recordCount) != null) { + String value = vv.getAccessor().getObject(recordCount).toString(); + partition[partitionColumnIndex] = value; + maxIndex = Math.max(maxIndex, partitionColumnIndex); + } + } + return Pair.of(partition, maxIndex); + } + protected LogicalExpression materializePruneExpr(RexNode pruneCondition, PlannerSettings settings, RelNode scanRel, diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java index b6e767e1b1b..e03cf22223a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java @@ -760,7 +760,6 @@ public List list(boolean recursive, Path... paths) throws IOExceptio } } - private void addRecursiveStatus(FileStatus parent, List listToFill) throws IOException { if (parent.isDir()) { Path pattern = new Path(parent.getPath(), "*"); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java index 5b4813ab6c4..8c7c0882bc6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.net.URI; -import java.util.BitSet; import java.util.List; import java.util.concurrent.TimeUnit; @@ -32,6 +31,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; + import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -47,16 +47,25 @@ public class FileSelection { private List statuses; public List files; + /** + * root path for the selections + */ public final String selectionRoot; + /** + * root path for the metadata cache file (if any) + */ + public final String cacheFileRoot; private enum StatusType { NOT_CHECKED, // initial state NO_DIRS, // no directories in this selection HAS_DIRS, // directories were found in the selection - EXPANDED // whether this selection has been expanded to files + EXPANDED_FULLY, // whether selection fully expanded to files + EXPANDED_PARTIAL // whether selection partially expanded to only directories (not files) } private StatusType dirStatus; + private boolean hadWildcard = false; // whether this selection previously had a wildcard /** * Creates a {@link FileSelection selection} out of given file statuses/files and selection root. @@ -66,10 +75,20 @@ private enum StatusType { * @param selectionRoot root path for selections */ public FileSelection(final List statuses, final List files, final String selectionRoot) { + this(statuses, files, selectionRoot, null, StatusType.NOT_CHECKED); + } + + public FileSelection(final List statuses, final List files, final String selectionRoot, final String cacheFileRoot) { + this(statuses, files, selectionRoot, cacheFileRoot, StatusType.NOT_CHECKED); + } + + public FileSelection(final List statuses, final List files, final String selectionRoot, + final String cacheFileRoot, final StatusType dirStatus) { this.statuses = statuses; this.files = files; this.selectionRoot = Preconditions.checkNotNull(selectionRoot); - this.dirStatus = StatusType.NOT_CHECKED; + this.dirStatus = dirStatus; + this.cacheFileRoot = cacheFileRoot; } /** @@ -81,6 +100,7 @@ protected FileSelection(final FileSelection selection) { this.files = selection.files; this.selectionRoot = selection.selectionRoot; this.dirStatus = selection.dirStatus; + this.cacheFileRoot = selection.cacheFileRoot; } public String getSelectionRoot() { @@ -128,7 +148,7 @@ public boolean containsDirectories(DrillFileSystem fs) throws IOException { } public FileSelection minusDirectories(DrillFileSystem fs) throws IOException { - if (isExpanded()) { + if (isExpandedFully()) { return this; } Stopwatch timer = Stopwatch.createStarted(); @@ -152,7 +172,7 @@ public boolean apply(@Nullable FileStatus status) { // fileSel will be null if we query an empty folder if (fileSel != null) { - fileSel.setExpanded(); + fileSel.setExpandedFully(); } return fileSel; @@ -162,12 +182,24 @@ public FileStatus getFirstPath(DrillFileSystem fs) throws IOException { return getStatuses(fs).get(0); } - public void setExpanded() { - this.dirStatus = StatusType.EXPANDED; + public void setExpandedFully() { + this.dirStatus = StatusType.EXPANDED_FULLY; + } + + public boolean isExpandedFully() { + return dirStatus == StatusType.EXPANDED_FULLY; + } + + public void setExpandedPartial() { + this.dirStatus = StatusType.EXPANDED_PARTIAL; + } + + public boolean isExpandedPartial() { + return dirStatus == StatusType.EXPANDED_PARTIAL; } - public boolean isExpanded() { - return dirStatus == StatusType.EXPANDED; + public StatusType getDirStatus() { + return dirStatus; } private static String commonPath(final List statuses) { @@ -229,13 +261,16 @@ private static String buildPath(final String[] path, final int folderIndex) { public static FileSelection create(final DrillFileSystem fs, final String parent, final String path) throws IOException { Stopwatch timer = Stopwatch.createStarted(); + boolean hasWildcard = path.contains(WILD_CARD); + final Path combined = new Path(parent, removeLeadingSlash(path)); - final FileStatus[] statuses = fs.globStatus(combined); + final FileStatus[] statuses = fs.globStatus(combined); // note: this would expand wildcards if (statuses == null) { return null; } final FileSelection fileSel = create(Lists.newArrayList(statuses), null, combined.toUri().toString()); logger.debug("FileSelection.create() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS)); + fileSel.setHadWildcard(hasWildcard); return fileSel; } @@ -275,6 +310,34 @@ public static FileSelection create(final List statuses, final List dirPaths, final FileSelection selection) { + final String root = selection.getSelectionRoot(); + if (Strings.isNullOrEmpty(root)) { + throw new DrillRuntimeException("Selection root is null or empty" + root); + } + if (dirPaths == null || dirPaths.isEmpty()) { + throw new DrillRuntimeException("List of directories is null or empty"); + } + + List dirs = Lists.newArrayList(); + + if (selection.hadWildcard()) { // for wildcard the directory list should have already been expanded + for (FileStatus status : selection.getFileStatuses()) { + dirs.add(status.getPath().toString()); + } + } else { + for (String s : dirPaths) { + dirs.add(s); + } + } + + final Path rootPath = handleWildCard(root); + // final URI uri = dirPaths.get(0).toUri(); + final URI uri = selection.getFileStatuses().get(0).getPath().toUri(); + final Path path = new Path(uri.getScheme(), uri.getAuthority(), rootPath.toUri().getPath()); + return new FileSelection(null, dirs, path.toString()); + } + private static Path handleWildCard(final String root) { if (root.contains(WILD_CARD)) { int idx = root.indexOf(WILD_CARD); // first wild card in the path @@ -300,7 +363,15 @@ public List getFileStatuses() { } public boolean supportDirPrunig() { - return isExpanded(); // currently we only support pruning if the directories have been expanded (this may change in the future) + return isExpandedFully() || isExpandedPartial(); + } + + public void setHadWildcard(boolean wc) { + this.hadWildcard = wc; + } + + public boolean hadWildcard() { + return this.hadWildcard; } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java index 7a80db3199b..409f80d56d2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java @@ -265,7 +265,7 @@ public GroupScan clone(List columns) { } @Override - public FileGroupScan clone(FileSelection selection) throws IOException { + public FileGroupScan clone(FileSelection selection, String cacheFileRoot /* ignored */) throws IOException { EasyGroupScan newScan = new EasyGroupScan(this); newScan.initFromSelection(selection, formatPlugin); newScan.mappings = null; /* the mapping will be created later when we get specific scan diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java index d7d31e52157..45f7ca2f9a3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java @@ -48,6 +48,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import org.apache.parquet.schema.Type; import org.codehaus.jackson.annotate.JsonIgnore; +import org.apache.commons.lang3.tuple.Pair; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; @@ -76,6 +77,7 @@ public class Metadata { public static final String[] OLD_METADATA_FILENAMES = {".drill.parquet_metadata.v2"}; public static final String METADATA_FILENAME = ".drill.parquet_metadata"; + public static final String METADATA_DIRECTORIES_FILENAME = ".drill.parquet_metadata_directories"; private final FileSystem fs; @@ -132,6 +134,11 @@ public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path) return metadata.readBlockMeta(path); } + public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path) throws IOException { + Metadata metadata = new Metadata(fs); + return metadata.readMetadataDirs(path); + } + private Metadata(FileSystem fs) { this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf()); } @@ -142,7 +149,8 @@ private Metadata(FileSystem fs) { * @param path * @throws IOException */ - private ParquetTableMetadata_v2 createMetaFilesRecursively(final String path) throws IOException { + private Pair + createMetaFilesRecursively(final String path) throws IOException { List metaDataList = Lists.newArrayList(); List directoryList = Lists.newArrayList(); ConcurrentHashMap columnTypeInfoSet = @@ -155,7 +163,7 @@ private ParquetTableMetadata_v2 createMetaFilesRecursively(final String path) th for (final FileStatus file : fs.listStatus(p, new DrillPathFilter())) { if (file.isDirectory()) { - ParquetTableMetadata_v2 subTableMetadata = createMetaFilesRecursively(file.getPath().toString()); + ParquetTableMetadata_v2 subTableMetadata = (createMetaFilesRecursively(file.getPath().toString())).getLeft(); metaDataList.addAll(subTableMetadata.files); directoryList.addAll(subTableMetadata.directories); directoryList.add(file.getPath().toString()); @@ -187,7 +195,14 @@ private ParquetTableMetadata_v2 createMetaFilesRecursively(final String path) th fs.delete(new Path(p, oldname), false); } writeFile(parquetTableMetadata, new Path(p, METADATA_FILENAME)); - return parquetTableMetadata; + + if (directoryList.size() > 0 && childFiles.size() == 0) { + ParquetTableMetadataDirs parquetTableMetadataDirs = new ParquetTableMetadataDirs(directoryList); + writeFile(parquetTableMetadataDirs, new Path(p, METADATA_DIRECTORIES_FILENAME)); + return Pair.of(parquetTableMetadata, parquetTableMetadataDirs); + } + List emptyDirList = Lists.newArrayList(); + return Pair.of(parquetTableMetadata, new ParquetTableMetadataDirs(emptyDirList)); } /** @@ -418,6 +433,19 @@ private void writeFile(ParquetTableMetadata_v2 parquetTableMetadata, Path p) thr os.close(); } + private void writeFile(ParquetTableMetadataDirs parquetTableMetadataDirs, Path p) throws IOException { + JsonFactory jsonFactory = new JsonFactory(); + jsonFactory.configure(Feature.AUTO_CLOSE_TARGET, false); + jsonFactory.configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, false); + ObjectMapper mapper = new ObjectMapper(jsonFactory); + SimpleModule module = new SimpleModule(); + mapper.registerModule(module); + FSDataOutputStream os = fs.create(p); + mapper.writerWithDefaultPrettyPrinter().writeValue(os, parquetTableMetadataDirs); + os.flush(); + os.close(); + } + /** * Read the parquet metadata from a file * @@ -447,11 +475,38 @@ private ParquetTableMetadataBase readBlockMeta(String path) throws IOException { timer.stop(); if (tableModified(parquetTableMetadata, p)) { parquetTableMetadata = - createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString()); + (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString())).getLeft(); } return parquetTableMetadata; } + private ParquetTableMetadataDirs readMetadataDirs(String path) throws IOException { + Stopwatch timer = Stopwatch.createStarted(); + Path p = new Path(path); + ObjectMapper mapper = new ObjectMapper(); + + final SimpleModule serialModule = new SimpleModule(); + serialModule.addDeserializer(SchemaPath.class, new SchemaPath.De()); + + AfterburnerModule module = new AfterburnerModule(); + module.setUseOptimizedBeanDeserializer(true); + + mapper.registerModule(serialModule); + mapper.registerModule(module); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + FSDataInputStream is = fs.open(p); + + ParquetTableMetadataDirs parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class); + logger.info("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + + if (tableModified(parquetTableMetadataDirs, p)) { + parquetTableMetadataDirs = + (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString())).getRight(); + } + return parquetTableMetadataDirs; + } + /** * Check if the parquet metadata needs to be updated by comparing the modification time of the directories with * the modification time of the metadata file @@ -477,6 +532,22 @@ private boolean tableModified(ParquetTableMetadataBase tableMetadata, Path metaF return false; } + private boolean tableModified(ParquetTableMetadataDirs tableMetadataDirs, Path metaFilePath) + throws IOException { + long metaFileModifyTime = fs.getFileStatus(metaFilePath).getModificationTime(); + FileStatus directoryStatus = fs.getFileStatus(metaFilePath.getParent()); + if (directoryStatus.getModificationTime() > metaFileModifyTime) { + return true; + } + for (String directory : tableMetadataDirs.getDirectories()) { + directoryStatus = fs.getFileStatus(new Path(directory)); + if (directoryStatus.getModificationTime() > metaFileModifyTime) { + return true; + } + } + return false; + } + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "metadata_version") @JsonSubTypes({ @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name="v1"), @@ -535,7 +606,22 @@ public static abstract class ColumnMetadata { public abstract OriginalType getOriginalType(); } + public static class ParquetTableMetadataDirs { + @JsonProperty List directories; + + public ParquetTableMetadataDirs() { + // default constructor needed for deserialization + } + public ParquetTableMetadataDirs(List directories) { + this.directories = directories; + } + + @JsonIgnore public List getDirectories() { + return directories; + } + + } @JsonTypeName("v1") public static class ParquetTableMetadata_v1 extends ParquetTableMetadataBase { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java index bf2e797b1f0..d7e7d825017 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java @@ -50,6 +50,7 @@ import org.apache.drill.exec.store.dfs.FormatSelection; import org.apache.drill.exec.store.dfs.MagicString; import org.apache.drill.exec.store.mock.MockStorageEngine; +import org.apache.drill.exec.store.parquet.Metadata.ParquetTableMetadataDirs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -164,7 +165,7 @@ public WriterRecordBatch getWriterBatch(FragmentContext context, RecordBatch inc @Override public ParquetGroupScan getGroupScan(String userName, FileSelection selection, List columns) throws IOException { - return new ParquetGroupScan(userName, selection, this, selection.selectionRoot, columns); + return new ParquetGroupScan(userName, selection, this, selection.selectionRoot, selection.cacheFileRoot, columns); } @Override @@ -208,8 +209,18 @@ public DrillTable isReadable(DrillFileSystem fs, FileSelection selection, FileSystemPlugin fsPlugin, String storageEngineName, String userName) throws IOException { // TODO: we only check the first file for directory reading. - if(selection.containsDirectories(fs)){ - if(isDirReadable(fs, selection.getFirstPath(fs))){ + if(selection.containsDirectories(fs)) { + Path dirMetaPath = new Path(selection.getSelectionRoot(), Metadata.METADATA_DIRECTORIES_FILENAME); + if (fs.exists(dirMetaPath)) { + ParquetTableMetadataDirs mDirs = Metadata.readMetadataDirs(fs, dirMetaPath.toString()); + if (mDirs.getDirectories().size() > 0) { + FileSelection dirSelection = FileSelection.createFromDirectories(mDirs.getDirectories(), selection); + dirSelection.setExpandedPartial(); + return new DynamicDrillTable(fsPlugin, storageEngineName, userName, + new FormatSelection(plugin.getConfig(), dirSelection)); + } + } + if(isDirReadable(fs, selection.getFirstPath(fs))) { return new DynamicDrillTable(fsPlugin, storageEngineName, userName, new FormatSelection(plugin.getConfig(), selection)); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java index 5950b74caf1..b5fc17b7604 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java @@ -111,12 +111,8 @@ public class ParquetGroupScan extends AbstractFileGroupScan { private List columns; private ListMultimap mappings; private List rowGroupInfos; - /** - * The parquet table metadata may have already been read - * from a metadata cache file earlier; we can re-use during - * the ParquetGroupScan and avoid extra loading time. - */ private Metadata.ParquetTableMetadataBase parquetTableMetadata = null; + private String cacheFileRoot = null; /* * total number of rows (obtained from parquet footer) @@ -135,7 +131,8 @@ public class ParquetGroupScan extends AbstractFileGroupScan { @JsonProperty("format") FormatPluginConfig formatConfig, // @JacksonInject StoragePluginRegistry engineRegistry, // @JsonProperty("columns") List columns, // - @JsonProperty("selectionRoot") String selectionRoot // + @JsonProperty("selectionRoot") String selectionRoot, // + @JsonProperty("cacheFileRoot") String cacheFileRoot // ) throws IOException, ExecutionSetupException { super(ImpersonationUtil.resolveUserName(userName)); this.columns = columns; @@ -150,6 +147,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan { this.formatConfig = formatPlugin.getConfig(); this.entries = entries; this.selectionRoot = selectionRoot; + this.cacheFileRoot = cacheFileRoot; init(); } @@ -159,6 +157,7 @@ public ParquetGroupScan( // FileSelection selection, // ParquetFormatPlugin formatPlugin, // String selectionRoot, + String cacheFileRoot, List columns) // throws IOException { super(userName); @@ -168,13 +167,13 @@ public ParquetGroupScan( // this.fs = ImpersonationUtil.createFileSystem(userName, formatPlugin.getFsConf()); this.selectionRoot = selectionRoot; + this.cacheFileRoot = cacheFileRoot; final FileSelection fileSelection = expandIfNecessary(selection); this.entries = Lists.newArrayList(); - final List files = fileSelection.getStatuses(fs); - for (FileStatus file : files) { - entries.add(new ReadEntryWithPath(file.getPath().toString())); + for (String fileName : fileSelection.getFiles()) { + entries.add(new ReadEntryWithPath(fileName)); } init(); @@ -201,6 +200,7 @@ private ParquetGroupScan(ParquetGroupScan that) { this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet); this.usedMetadataCache = that.usedMetadataCache; this.parquetTableMetadata = that.parquetTableMetadata; + this.cacheFileRoot = that.cacheFileRoot; } /** @@ -213,16 +213,18 @@ private ParquetGroupScan(ParquetGroupScan that) { * @throws IOException */ private FileSelection expandIfNecessary(FileSelection selection) throws IOException { - if (selection.isExpanded()) { + if (selection.isExpandedFully()) { return selection; } - Path metaFilePath = new Path(selection.getSelectionRoot(), Metadata.METADATA_FILENAME); + // use the cacheFileRoot if provided (e.g after partition pruning) + Path metaFilePath = new Path(cacheFileRoot != null ? cacheFileRoot : selectionRoot, Metadata.METADATA_FILENAME); if (!fs.exists(metaFilePath)) { // no metadata cache return selection; } - return initFromMetadataCache(selection, metaFilePath); + FileSelection expandedSelection = initFromMetadataCache(selection, metaFilePath); + return expandedSelection; } public List getEntries() { @@ -549,7 +551,6 @@ public long getRowCount() { } - /** * Create and return a new file selection based on reading the metadata cache file. * @@ -580,6 +581,12 @@ public long getRowCount() { fileNames.add(file.getPath()); } // we don't need to populate fileSet as all files are selected + } else if (selection.isExpandedPartial() && cacheFileRoot != null) { + this.parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath.toString()); + for (Metadata.ParquetFileMetadata file : this.parquetTableMetadata.getFiles()) { + fileNames.add(file.getPath()); + } + fileSet = Sets.newHashSet(fileNames); } else { // we need to expand the files from fileStatuses for (FileStatus status : fileStatuses) { @@ -616,9 +623,9 @@ public long getRowCount() { // because create() changes the root to include the scheme and authority; In future, if create() // is the preferred way to instantiate a file selection, we may need to do something different... // WARNING: file statuses and file names are inconsistent - FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString()); + FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString(), cacheFileRoot); - newSelection.setExpanded(); + newSelection.setExpandedFully(); return newSelection; } @@ -837,10 +844,20 @@ public String getDigest() { @Override public String toString() { + String cacheFileString = ""; + if (usedMetadataCache) { + // for EXPLAIN it is useful to know what was the root of the cache file; + // if the cacheFileRoot is null (depending on whether single partition + // optimization was applied), then we would have read the cache file from + // selectionRoot + String str = (cacheFileRoot == null) ? selectionRoot : cacheFileRoot; + cacheFileString = ", cacheFileRoot=" + str; + } return "ParquetGroupScan [entries=" + entries + ", selectionRoot=" + selectionRoot + ", numFiles=" + getEntries().size() + ", usedMetadataFile=" + usedMetadataCache + + cacheFileString + ", columns=" + columns + "]"; } @@ -852,9 +869,10 @@ public GroupScan clone(List columns) { } @Override - public FileGroupScan clone(FileSelection selection) throws IOException { + public FileGroupScan clone(FileSelection selection, String cacheFileRoot) throws IOException { ParquetGroupScan newScan = new ParquetGroupScan(this); newScan.modifyFileSelection(selection); + newScan.cacheFileRoot = cacheFileRoot; newScan.init(); return newScan; } @@ -895,7 +913,7 @@ public GroupScan applyLimit(long maxRecords) { try { FileSelection newSelection = new FileSelection(null, Lists.newArrayList(fileNames), getSelectionRoot()); logger.debug("applyLimit() reduce parquet file # from {} to {}", fileSet.size(), fileNames.size()); - return this.clone(newSelection); + return this.clone(newSelection, this.cacheFileRoot); } catch (IOException e) { logger.warn("Could not apply rowcount based prune due to Exception : {}", e); return null; 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 afcea879b8b..f6225e00205 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 @@ -47,24 +47,28 @@ public static void copyData() throws Exception { dataDir); } - @Test + @Test // also a negative test case for DRILL-4530 public void testPartitionPruningWithMetadataCache_1() throws Exception { test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName)); checkForMetadataFile(tableName); String query = String.format("select dir0, dir1, o_custkey, o_orderdate from dfs_test.`%s/%s` " + - " where dir0=1994 and dir1='Q1'", + " where dir0=1994 and dir1 in ('Q1', 'Q2')", getDfsTestTmpSchemaLocation(), tableName); - int expectedRowCount = 10; - int expectedNumFiles = 1; + int expectedRowCount = 20; + int expectedNumFiles = 2; int actualRowCount = testSql(query); assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern}, new String[] {"Filter"}); + // since there are 2 or more sub-partitions the single partition cache file optimization does not apply + // and cacheFileRoot should point to the top level selectionRoot + String cacheFileRootPattern = String.format("%s/%s", getDfsTestTmpSchemaLocation(), tableName); + PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, + new String[] {"Filter"}); } - @Test // DRILL-3917 + @Test // DRILL-3917, positive test case for DRILL-4530 public void testPartitionPruningWithMetadataCache_2() throws Exception { test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName)); checkForMetadataFile(tableName); @@ -78,7 +82,9 @@ public void testPartitionPruningWithMetadataCache_2() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern}, new String[] {"Filter"}); + String cacheFileRootPattern = String.format("%s/%s/1994", getDfsTestTmpSchemaLocation(), tableName); + PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, + new String[] {"Filter"}); } @Test // DRILL-3937 (partitioning column is varchar) @@ -98,8 +104,8 @@ public void testPartitionPruningWithMetadataCache_3() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - - testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern}, new String[] {}); + testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern}, + new String[] {}); } @Test // DRILL-3937 (partitioning column is binary using convert_to) From e80c65b033b329c0e0333464fac5c3fd258ce163 Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Mon, 27 Jun 2016 16:00:30 -0700 Subject: [PATCH 2/7] Check if prefix components are non-null the very first time single partition info is initialized. --- .../exec/planner/logical/partition/PruneScanRule.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) 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 31be896cbe1..7e4c96f8bbf 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 @@ -302,6 +302,12 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR String[] parts = p.getLeft(); int tmpIndex = p.getRight(); if (spInfo == null) { + for (int j = 0; j <= tmpIndex; j++) { + if (parts[j] == null) { // prefixes should be non-null + isSinglePartition = false; + break; + } + } spInfo = parts; maxIndex = tmpIndex; } else if (maxIndex != tmpIndex) { @@ -310,8 +316,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR } else { // we only want to compare until the maxIndex inclusive since subsequent values would be null for (int j = 0; j <= maxIndex; j++) { - if (spInfo[j] == null // prefixes should be non-null - || !spInfo[j].equals(parts[j])) { + if (!spInfo[j].equals(parts[j])) { isSinglePartition = false; break; } From 4f777992a6dddff027b54a730411e04154056623 Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Wed, 6 Jul 2016 23:45:12 -0700 Subject: [PATCH 3/7] Add separate interface method to create scan using a cacheFileRoot. --- .../exec/planner/sql/HivePartitionDescriptor.java | 2 +- .../exec/planner/AbstractPartitionDescriptor.java | 8 ++++++++ .../exec/planner/FileSystemPartitionDescriptor.java | 5 +++++ .../exec/planner/ParquetPartitionDescriptor.java | 6 ++++++ .../drill/exec/planner/PartitionDescriptor.java | 11 ++++++++++- .../exec/planner/logical/partition/PruneScanRule.java | 3 ++- 6 files changed, 32 insertions(+), 3 deletions(-) diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java index f0b68c1a95d..8a7e43a7eeb 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java @@ -152,7 +152,7 @@ protected void createPartitionSublists() { } @Override - public TableScan createTableScan(List newPartitions, String cacheFileRoot) throws Exception { + public TableScan createTableScan(List newPartitions) throws Exception { GroupScan newGroupScan = createNewGroupScan(newPartitions); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java index 66b26d8ee9c..49b00c4dc13 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java @@ -20,6 +20,8 @@ import java.util.Iterator; import java.util.List; +import org.apache.calcite.rel.core.TableScan; + /** * Abstract base class for file system based partition descriptors and Hive partition descriptors. * @@ -60,4 +62,10 @@ public boolean supportsSinglePartOptimization() { return false; } + + @Override + public TableScan createTableScan(List newPartitions, String cacheFileRoot) throws Exception { + throw new UnsupportedOperationException(); + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java index 6265e359bea..46738d146db 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java @@ -258,6 +258,11 @@ private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, L return EnumerableTableScan.create(oldScan.getCluster(), newOptTableImpl); } + @Override + public TableScan createTableScan(List newPartitionLocation) throws Exception { + return createTableScan(newPartitionLocation, null); + } + @Override public boolean supportsSinglePartOptimization() { return true; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java index ccc5e156604..7ed36af3147 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java @@ -147,4 +147,10 @@ public TableScan createTableScan(List newPartitionLocation, S scanRel.getColumns(), true /*filter pushdown*/); } + + @Override + public TableScan createTableScan(List newPartitionLocation) throws Exception { + return createTableScan(newPartitionLocation, null); + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java index a9410b56ae8..50c160bcd47 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java @@ -77,7 +77,16 @@ void populatePartitionVectors(ValueVector[] vectors, List par /** * Methods create a new TableScan rel node, given the lists of new partitions or new files to SCAN. * @param newPartitions - * @param cacheFileRoot - optional (can be null) + * @return + * @throws Exception + */ + public TableScan createTableScan(List newPartitions) throws Exception; + + /** + * Create a new TableScan rel node, given the lists of new partitions or new files to scan and a path + * to a metadata cache file + * @param newPartitions + * @param cacheFileRoot * @return * @throws Exception */ 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 7e4c96f8bbf..8a4879036c0 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 @@ -392,7 +392,8 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR cacheFileRoot = descriptor.getBaseTableLocation() + path; } - RelNode inputRel = descriptor.createTableScan(newPartitions, cacheFileRoot); + RelNode inputRel = descriptor.supportsSinglePartOptimization() ? + descriptor.createTableScan(newPartitions, cacheFileRoot) : descriptor.createTableScan(newPartitions); if (projectRel != null) { inputRel = projectRel.copy(projectRel.getTraitSet(), Collections.singletonList(inputRel)); From 2bc7fbcd19287f1124dd1cfb0771910c9e05600f Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Tue, 12 Jul 2016 12:57:19 -0700 Subject: [PATCH 4/7] Create filenames list with unique names using fileSet if available. Add several unit tests. --- .../exec/store/parquet/ParquetGroupScan.java | 11 +- .../parquet/TestParquetMetadataCache.java | 97 +++++++++++++++--- .../multilevel/parquet2/1994/Q1/1.parquet | Bin 0 -> 2015 bytes .../multilevel/parquet2/1994/Q1/2.parquet | Bin 0 -> 2015 bytes .../multilevel/parquet2/1994/Q2/1.parquet | Bin 0 -> 2130 bytes .../multilevel/parquet2/1994/Q2/2.parquet | Bin 0 -> 2130 bytes .../multilevel/parquet2/1994/Q3/1.parquet | Bin 0 -> 2054 bytes .../multilevel/parquet2/1994/Q3/2.parquet | Bin 0 -> 2054 bytes .../multilevel/parquet2/1994/Q4/1.parquet | Bin 0 -> 2056 bytes .../multilevel/parquet2/1994/Q4/2.parquet | Bin 0 -> 2056 bytes .../multilevel/parquet2/1995/Q1/1.parquet | Bin 0 -> 2180 bytes .../multilevel/parquet2/1995/Q1/2.parquet | Bin 0 -> 2180 bytes .../multilevel/parquet2/1995/Q2/1.parquet | Bin 0 -> 2110 bytes .../multilevel/parquet2/1995/Q2/2.parquet | Bin 0 -> 2110 bytes .../multilevel/parquet2/1995/Q3/1.parquet | Bin 0 -> 1902 bytes .../multilevel/parquet2/1995/Q3/2.parquet | Bin 0 -> 1902 bytes .../multilevel/parquet2/1995/Q4/1.parquet | Bin 0 -> 2013 bytes .../multilevel/parquet2/1995/Q4/2.parquet | Bin 0 -> 2013 bytes 18 files changed, 94 insertions(+), 14 deletions(-) create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q1/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q1/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q2/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q2/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q3/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q3/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q4/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q4/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q1/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q1/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q2/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q2/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q3/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q3/2.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q4/1.parquet create mode 100644 exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q4/2.parquet diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java index b5fc17b7604..4973beac581 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java @@ -607,7 +607,14 @@ public long getRowCount() { fileSet = Sets.newHashSet(fileNames); } - if (fileNames.isEmpty()) { + List finalFileNames; + if (fileSet != null) { + finalFileNames = Lists.newArrayList(fileSet); + } else { + finalFileNames = fileNames; + } + + if (finalFileNames.isEmpty()) { // no files were found, most likely we tried to query some empty sub folders throw UserException.validationError().message("The table you tried to query is empty").build(logger); } @@ -623,7 +630,7 @@ public long getRowCount() { // because create() changes the root to include the scheme and authority; In future, if create() // is the preferred way to instantiate a file selection, we may need to do something different... // WARNING: file statuses and file names are inconsistent - FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString(), cacheFileRoot); + FileSelection newSelection = new FileSelection(selection.getStatuses(fs), finalFileNames, metaRootPath.toString(), cacheFileRoot); newSelection.setExpandedFully(); return newSelection; 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 f6225e00205..0afd9259619 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 @@ -34,26 +34,32 @@ public class TestParquetMetadataCache extends PlanTestBase { private static final String WORKING_PATH = TestTools.getWorkingPath(); private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources"; - private static final String tableName = "parquetTable"; + private static final String tableName1 = "parquetTable1"; + private static final String tableName2 = "parquetTable2"; @BeforeClass public static void copyData() throws Exception { // copy the data into the temporary location String tmpLocation = getDfsTestTmpSchemaLocation(); - File dataDir = new File(tmpLocation + Path.SEPARATOR + tableName); - dataDir.mkdir(); + File dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1); + dataDir1.mkdir(); FileUtils.copyDirectory(new File(String.format(String.format("%s/multilevel/parquet", TEST_RES_PATH))), - dataDir); + dataDir1); + + File dataDir2 = new File(tmpLocation + Path.SEPARATOR + tableName2); + dataDir2.mkdir(); + FileUtils.copyDirectory(new File(String.format(String.format("%s/multilevel/parquet2", TEST_RES_PATH))), + dataDir2); } @Test // also a negative test case for DRILL-4530 public void testPartitionPruningWithMetadataCache_1() throws Exception { - test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName)); - checkForMetadataFile(tableName); + 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 and dir1 in ('Q1', 'Q2')", - getDfsTestTmpSchemaLocation(), tableName); + getDfsTestTmpSchemaLocation(), tableName1); int expectedRowCount = 20; int expectedNumFiles = 2; @@ -63,18 +69,18 @@ public void testPartitionPruningWithMetadataCache_1() throws Exception { String usedMetaPattern = "usedMetadataFile=true"; // since there are 2 or more sub-partitions the single partition cache file optimization does not apply // and cacheFileRoot should point to the top level selectionRoot - String cacheFileRootPattern = String.format("%s/%s", getDfsTestTmpSchemaLocation(), tableName); + String cacheFileRootPattern = String.format("%s/%s", getDfsTestTmpSchemaLocation(), tableName1); PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } @Test // DRILL-3917, positive test case for DRILL-4530 public void testPartitionPruningWithMetadataCache_2() throws Exception { - test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName)); - checkForMetadataFile(tableName); + 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", - getDfsTestTmpSchemaLocation(), tableName); + getDfsTestTmpSchemaLocation(), tableName1); int expectedRowCount = 40; int expectedNumFiles = 4; @@ -82,7 +88,7 @@ public void testPartitionPruningWithMetadataCache_2() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - String cacheFileRootPattern = String.format("%s/%s/1994", getDfsTestTmpSchemaLocation(), tableName); + String cacheFileRootPattern = String.format("%s/%s/1994", getDfsTestTmpSchemaLocation(), tableName1); PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } @@ -211,9 +217,76 @@ public void testNoSupportedError() throws Exception { .go(); } + @Test // DRILL-4530 + public void testDrill4530_1() throws Exception { + // create metadata cache + test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName2)); + checkForMetadataFile(tableName2); + + // run query and check correctness + String query1 = String.format("select dir0, dir1, o_custkey, o_orderdate from dfs_test.`%s/%s` " + + " where dir0=1995 and dir1='Q3'", + getDfsTestTmpSchemaLocation(), tableName2); + int expectedRowCount = 20; + int expectedNumFiles = 2; + + int actualRowCount = testSql(query1); + assertEquals(expectedRowCount, actualRowCount); + String numFilesPattern = "numFiles=" + expectedNumFiles; + String usedMetaPattern = "usedMetadataFile=true"; + String cacheFileRootPattern = String.format("%s/%s/1995/Q3", getDfsTestTmpSchemaLocation(), tableName2); + PlanTestBase.testPlanMatchingPatterns(query1, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, + new String[] {"Filter"}); + } + + @Test // DRILL-4530 + public void testDrill4530_2() throws Exception { + // create metadata cache + test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName2)); + checkForMetadataFile(tableName2); + + // run query and check correctness + String query1 = String.format("select dir0, dir1, o_custkey, o_orderdate from dfs_test.`%s/%s` " + + " where dir0=1995", + getDfsTestTmpSchemaLocation(), tableName2); + int expectedRowCount = 80; + int expectedNumFiles = 8; + + int actualRowCount = testSql(query1); + assertEquals(expectedRowCount, actualRowCount); + String numFilesPattern = "numFiles=" + expectedNumFiles; + String usedMetaPattern = "usedMetadataFile=true"; + String cacheFileRootPattern = String.format("%s/%s/1995", getDfsTestTmpSchemaLocation(), tableName2); + PlanTestBase.testPlanMatchingPatterns(query1, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, + new String[] {"Filter"}); + } + + @Test // DRILL-4530 + public void testDrill4530_3() throws Exception { + // create metadata cache + test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName2)); + checkForMetadataFile(tableName2); + + // run query and check correctness + String query1 = String.format("select dir0, dir1, o_custkey, o_orderdate from dfs_test.`%s/%s` " + + " where dir1='Q3'", + getDfsTestTmpSchemaLocation(), tableName2); + int expectedRowCount = 40; + int expectedNumFiles = 4; + + int actualRowCount = testSql(query1); + assertEquals(expectedRowCount, actualRowCount); + String numFilesPattern = "numFiles=" + expectedNumFiles; + String usedMetaPattern = "usedMetadataFile=true"; + String cacheFileRootPattern = String.format("%s/%s", getDfsTestTmpSchemaLocation(), tableName2); + PlanTestBase.testPlanMatchingPatterns(query1, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, + new String[] {"Filter"}); + } + private void checkForMetadataFile(String table) throws Exception { String tmpDir = getDfsTestTmpSchemaLocation(); String metaFile = Joiner.on("/").join(tmpDir, table, Metadata.METADATA_FILENAME); Assert.assertTrue(Files.exists(new File(metaFile).toPath())); } + } diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q1/1.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q1/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b4abe60c6ff652ebe4af8c2d2d31b0c30e14415b GIT binary patch literal 2015 zcmcIlU2NM_6u!=q;slqrYiv1MGWRhsqyXh<+s?KCtb{-%Kt4}C(Dc!t;Q328iT8X6L-1cHilouqR+#v4(NeeXHn zcmA$@Jk@vlD3AClemc&FVZrijRP^`lLrC=ZBZLGH6kDfD735_Gp*D~Kkk>(0K|Z14 zLx_@|b+rvZOFiKU z?Ei9O97;V4*J`Pf2FneQvyJR|YP<%r3bF*U2C@tSZnhczwzq;nSK`FUb0T{*apv^M z(B!o1?Oh#ug^l>qVv3N%+k_*_9EFI&rGz-i*tXGn-uh23sfCtNx6jddTF$kP_+<1VSC3+z_i%^p9 z8VBg11AdzX_P2a3DpqBo4C^1oqGl+1T*`ckIAxYN!J(GV znCrl#U5@x>aj0MAZ4hj>bE zs_6hmOwF!PI@P}bLFj8j_cV)_$c#0omFuu7h-N|-7O-IhObAD!k#(7kLJ94Njd&C1 zeL`5c$_O7ZGMjs5{L0sv9n#a{Je=zckoF8(`8!{f=cq2CV$MeWRhsqyXh<+s?KCtb{-%Kt4}C(Dc!t;Q328iT8X6L-1cHilouqR+#v4(NeeXHn zcmA$@Jk@vlD3AClemc&FVZrijRP^`lLrC=ZBZLGH6kDfD735_Gp*D~Kkk>(0K|Z14 zLx_@|b+rvZOFiKU z?Ei9O97;V4*J`Pf2FneQvyJR|YP<%r3bF*U2C@tSZnhczwzq;nSK`FUb0T{*apv^M z(B!o1?Oh#ug^l>qVv3N%+k_*_9EFI&rGz-i*tXGn-uh23sfCtNx6jddTF$kP_+<1VSC3+z_i%^p9 z8VBg11AdzX_P2a3DpqBo4C^1oqGl+1T*`ckIAxYN!J(GV znCrl#U5@x>aj0MAZ4hj>bE zs_6hmOwF!PI@P}bLFj8j_cV)_$c#0omFuu7h-N|-7O-IhObAD!k#(7kLJ94Njd&C1 zeL`5c$_O7ZGMjs5{L0sv9n#a{Je=zckoF8(`8!{f=cq2CV$Mev z0HJj-LE1015fV}{;8dli0M-`-3XH=2KFl|OE`WDnJ^*$VWI-w~N9&@(qbeTtMY>vH z9gTh(5}D0ADXAfSA@L^?uphww1QWntVHD6iU?6oaAg|S@5$a7%o_tMh>`IM|A0P8GID^+) zAn%^kiIX$rmFbaVOq2&0M(VX3>jW9T3X4)hMfv#)Cuo;_WI8j^q^ zT)o{x8j65k%iviEe`o+nogIAw(BIMPS20p|esKKa#)zlCT>_z1EGo3sB!AIZ2h6!? z;<6&;W|6wYq4S=lJ1{PJ)a5QQ^f#GB)3t>pv1uu9mgb3W7|bgHLRTkAp4xl?rkvS? zFHquIWvdJ`vm}<7YXUA$IW;)ZiwxKTRPY>=c>pD@Y0(la2}wWNdkxEz5c$98;%A5V z%ZB3o(V{YlG3N5vbD1=;%@Sux@aP3jo#vyhmRBg4IAhz?Fm;O**fLZCKy=>GIV5u! zuudr1y5yP0;!__ELzE7OvKhKV3!Y__pT}Qq&D08GmR#<51~*v=Fmsj(xk8R!DF-tD zwb%z{kdALf{W`yZZxWZhtuIodTb)v5j#+s*{$85Uw`eIz+Ce@=Oox?BgUp)+DES<< zOjslX{ZT`I5!MIKqoD4j4ah!)%s0dARblTyZqBS;KR={~iIrfhxe|I``Z- zG^H?jBW#v4u3NTa8ni-uq6*AI+2au;3vx@<2QcQCye#O#-vpb8@0oZPQ9Q|J-38OG z!73mT2`d)iasf<<#@b?Qn$!j<#E}%M4_po?QRNa=KEj%m+1I(X%Xo+MFu6L;WelV} zJzD!b&?ZMJJMFH3Ph7>y1VA#-jcW~AdjO1Zg|VwTK4BXx_b@;*>f?H|wjJCFv%Zvd zfo=yO&^--S`=D_r(7o06sow;YpD~~^>Lc2zcDHGVT{{z>^;?5-AponH-Tm*e4N-eH zyc7MN!>lE60x%FiT;%%JrhN^>YM~z(vJ N&@B9(Ps6{7e*h`kAYcFh literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q2/2.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q2/2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f5338af3c1418e4c7a20ceed9ba6e1b6f906fb50 GIT binary patch literal 2130 zcmd5;&u<$=6rOSHWMfv z0HJj-LE1015fV}{;8dli0M-`-3XH=2KFl|OE`WDnJ^*$VWI-w~N9&@(qbeTtMY>vH z9gTh(5}D0ADXAfSA@L^?uphww1QWntVHD6iU?6oaAg|S@5$a7%o_tMh>`IM|A0P8GID^+) zAn%^kiIX$rmFbaVOq2&0M(VX3>jW9T3X4)hMfv#)Cuo;_WI8j^q^ zT)o{x8j65k%iviEe`o+nogIAw(BIMPS20p|esKKa#)zlCT>_z1EGo3sB!AIZ2h6!? z;<6&;W|6wYq4S=lJ1{PJ)a5QQ^f#GB)3t>pv1uu9mgb3W7|bgHLRTkAp4xl?rkvS? zFHquIWvdJ`vm}<7YXUA$IW;)ZiwxKTRPY>=c>pD@Y0(la2}wWNdkxEz5c$98;%A5V z%ZB3o(V{YlG3N5vbD1=;%@Sux@aP3jo#vyhmRBg4IAhz?Fm;O**fLZCKy=>GIV5u! zuudr1y5yP0;!__ELzE7OvKhKV3!Y__pT}Qq&D08GmR#<51~*v=Fmsj(xk8R!DF-tD zwb%z{kdALf{W`yZZxWZhtuIodTb)v5j#+s*{$85Uw`eIz+Ce@=Oox?BgUp)+DES<< zOjslX{ZT`I5!MIKqoD4j4ah!)%s0dARblTyZqBS;KR={~iIrfhxe|I``Z- zG^H?jBW#v4u3NTa8ni-uq6*AI+2au;3vx@<2QcQCye#O#-vpb8@0oZPQ9Q|J-38OG z!73mT2`d)iasf<<#@b?Qn$!j<#E}%M4_po?QRNa=KEj%m+1I(X%Xo+MFu6L;WelV} zJzD!b&?ZMJJMFH3Ph7>y1VA#-jcW~AdjO1Zg|VwTK4BXx_b@;*>f?H|wjJCFv%Zvd zfo=yO&^--S`=D_r(7o06sow;YpD~~^>Lc2zcDHGVT{{z>^;?5-AponH-Tm*e4N-eH zyc7MN!>lE60x%FiT;%%JrhN^>YM~z(vJ N&@B9(Ps6{7e*h`kAYcFh literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q3/1.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q3/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..16cb2c46c7168d8f0027fe3aa9bc03feeb3c3b40 GIT binary patch literal 2054 zcmcguO>7%Q6rOSHW@BuU6z_IZS!HR72`CP+^P@DF$Vv0#RH^Hhgdfo=Y}R9YmG#bc zXE)R!^?+1U?o)D3k6X$!y2y6sVh|2y}KSHuUfDn?vP|qsgroooL-UIs?>=BI6 z!#;#wZ9(X3FiyHN)m1N5H{ffjz5?0=m*>mtp0=r(t~_VjK>s@?xY(6`q#y zw3j5=0T?uWDZoMRVsRBW!Gy@iGuO_nhQ?)~0H4_S_i2xmd-m-0W9u({{&)JDn>{D~ zJaaA0!xvupEg+OQs7_noclP?X`+eh^$JY0K_74hl%pJ`Bl71P&Q}6tdKJn(2oBKC} zaj($1repnN?J)FgFML-EP4~c-KdN9EFcS8sPQuy^#@AUeNVAE>51RD{hT?NapOJ+` zeBt=aRCeCWwvyvF;6()348)I2P0lVH!J)}IY!|qb@yzT@CL2V3kD*LEI0oa{qw~sB zxv4|5)3Z~P_`$vf6s?ot?0KG5%d4km}>ZH`2;-xG5g_weL3Yyb;@ z9D4u>!@~)_3=WR)WgwZXJ`Ka-PYg{gBLCpP1nfr=6E+ec+t{3k_Iep=k^yf5fB*P* zwU-E8#JpEmTg6XyH354oMnx&;rfMiQIpq@OFomuVTX9N6DY&*yT?Po76kS7=M-@xe zmenF*N=Ypi4Wd90!fL^(Q8d*i1=j$dZc|g&6y0Qw?P`uEW3y#r7$lu?b#2+G^eIc! z$SZkbQKrMW%pk<7X6$PmX?OpyTw72}aAr(aRvFc9x}^4po+j zYBEJ#qORjrk*9`L(#;ek&=i$b%9cYN!uk}Df!~PMYO}lCc~+t>2r8*7M1e%6T7=*M zoZhGx^^n3WqUq4AdRaP!vzTsa;-7=Eiu#iMdvhn9FNJa=`vR}lHZ|- zWvI}E&JR2EJW*NrWM~g$>3ln?kkgh;7z^iK=mpPRuPFRl<9fo+h4yl$7F9Kkx~9Xh z+ctu;A@+o#KWe1mE-7?DhpH9Tpk@*8?pGQ&MNW&pu8%T)n^H&IjYMBJl7n=S+Ie`D zNu}y&ql=o$9Pc>l&#N@2dgxszbU%NaW=9nh#*q z)}0EcbAKxw!hb*fnX9;;F0ztt)nQc-S3=8WVmbhmB3;q0)tC^461tJl)ht}}NfGHh zmM-9!kh)_e$JX!`sZm@LXAJ{s$MB7P>Whk@>Pg!>;1Mrk=`cW2FlOtMHg+2r!75{0 zcRa!^Ed7oFl9KN=bYnOCTQZwlSr>W^0|JEZ7@o1)fi0msyX{fG^htlg$$^x7*KrxU z)w;#6--^fkwM9DbgJ^2|WV`36i@hG)(!b+6>m_6X0~Np<+&r>kKLD}T>c*9OoFChy kO9I4G^8Y)&q@VUk5!4g3R2xR66EE9n6#n++;6H%>0BwT(+5i9m literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q3/2.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q3/2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..16cb2c46c7168d8f0027fe3aa9bc03feeb3c3b40 GIT binary patch literal 2054 zcmcguO>7%Q6rOSHW@BuU6z_IZS!HR72`CP+^P@DF$Vv0#RH^Hhgdfo=Y}R9YmG#bc zXE)R!^?+1U?o)D3k6X$!y2y6sVh|2y}KSHuUfDn?vP|qsgroooL-UIs?>=BI6 z!#;#wZ9(X3FiyHN)m1N5H{ffjz5?0=m*>mtp0=r(t~_VjK>s@?xY(6`q#y zw3j5=0T?uWDZoMRVsRBW!Gy@iGuO_nhQ?)~0H4_S_i2xmd-m-0W9u({{&)JDn>{D~ zJaaA0!xvupEg+OQs7_noclP?X`+eh^$JY0K_74hl%pJ`Bl71P&Q}6tdKJn(2oBKC} zaj($1repnN?J)FgFML-EP4~c-KdN9EFcS8sPQuy^#@AUeNVAE>51RD{hT?NapOJ+` zeBt=aRCeCWwvyvF;6()348)I2P0lVH!J)}IY!|qb@yzT@CL2V3kD*LEI0oa{qw~sB zxv4|5)3Z~P_`$vf6s?ot?0KG5%d4km}>ZH`2;-xG5g_weL3Yyb;@ z9D4u>!@~)_3=WR)WgwZXJ`Ka-PYg{gBLCpP1nfr=6E+ec+t{3k_Iep=k^yf5fB*P* zwU-E8#JpEmTg6XyH354oMnx&;rfMiQIpq@OFomuVTX9N6DY&*yT?Po76kS7=M-@xe zmenF*N=Ypi4Wd90!fL^(Q8d*i1=j$dZc|g&6y0Qw?P`uEW3y#r7$lu?b#2+G^eIc! z$SZkbQKrMW%pk<7X6$PmX?OpyTw72}aAr(aRvFc9x}^4po+j zYBEJ#qORjrk*9`L(#;ek&=i$b%9cYN!uk}Df!~PMYO}lCc~+t>2r8*7M1e%6T7=*M zoZhGx^^n3WqUq4AdRaP!vzTsa;-7=Eiu#iMdvhn9FNJa=`vR}lHZ|- zWvI}E&JR2EJW*NrWM~g$>3ln?kkgh;7z^iK=mpPRuPFRl<9fo+h4yl$7F9Kkx~9Xh z+ctu;A@+o#KWe1mE-7?DhpH9Tpk@*8?pGQ&MNW&pu8%T)n^H&IjYMBJl7n=S+Ie`D zNu}y&ql=o$9Pc>l&#N@2dgxszbU%NaW=9nh#*q z)}0EcbAKxw!hb*fnX9;;F0ztt)nQc-S3=8WVmbhmB3;q0)tC^461tJl)ht}}NfGHh zmM-9!kh)_e$JX!`sZm@LXAJ{s$MB7P>Whk@>Pg!>;1Mrk=`cW2FlOtMHg+2r!75{0 zcRa!^Ed7oFl9KN=bYnOCTQZwlSr>W^0|JEZ7@o1)fi0msyX{fG^htlg$$^x7*KrxU z)w;#6--^fkwM9DbgJ^2|WV`36i@hG)(!b+6>m_6X0~Np<+&r>kKLD}T>c*9OoFChy kO9I4G^8Y)&q@VUk5!4g3R2xR66EE9n6#n++;6H%>0BwT(+5i9m literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q4/1.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1994/Q4/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..bf0ed058c96b9851b77be98a98a0bf5408bf3342 GIT binary patch literal 2056 zcmc&$O>7%Q6rORs$;LQ@67RA`Ry(MQn?PKgABS3Xi=8&VN|O*0AtBUgV~_16n;mC= zN*rAWC9P3EpTzNC}oOLNbC+*CjqY1Ca|Lv<~qq#8(hE zAWj4LG6W~xE45ePQb)F>M1V~P5$Xo&D%}5qrL%BvhPVb}PT+8mDo7DuRd__fBW^u? z55YPb`9$EL4={XvFmdHb=*QKQQce-Cj9y>cdj4A0CC&WQ{le>CzWr1v=S{=opXW~g zn)L|lH=nsWX|?3`!Jy~U+u7Nxd)ME*&@L#G7oHSy*ViIfzBBgZI4kze?X0`iSzqGd z=Nl8;HMrJXi+u*e^AK3nKZM)65br>697kAD=(b4CB7U*!bue7=~PYv?Cfi*pH;{v38gmNK8%7k!NRykByIv z4sSGc>HdME9Z2j; z4ZM(|jp4087E-;Ls#olS3vFA&Ki^*zY2UgNyuHa}lA7 zn3wxQi*&l#2Rv*tL(7u_(<^1ounrQSsDPTQBq5k zSTRqIEq>Pf>#u*(wFU zikhMJ;}^r+Dq%C|*m&X)u{50;cHP9no--oYBmEK1fkfN4k*8g!q@Q~`6tJ0DHi`a& z)KrIOPW1qOIa}XBmC)6SNuiFu3i6EC)A)&)XT5M%aKha6$L3K_n;6|6sTtj>g%3kM z;ehmB^pNmNH;UBGQL;$m1f@Xqqp_gLm@Pewq(CQ9LTrJV1$cvLwKmbp7V?f|yU$?& zwABHYt=f*&2^}T`vthPiGh5Xwrk1BQ;sbSHLA7aT7)hMmUY`RPGc~))>3qHglJK7m zf96#@$QG=!R`Fmp5U+$SFN3iFlOxes^io`m!4^7^81)w}1>}hQ2A0p`xR|-84aYZd zgS5RlA7=vtX;;IIZw6vgxRzOC0x6&*0Eer@KbXU8Kzt-F+bhq0s_3MEA8wONH`Ooecztz&^es7Y` z1z8Rh?MTk_pU Z7%Q6rORs$;LQ@67RA`Ry(MQn?PKgABS3Xi=8&VN|O*0AtBUgV~_16n;mC= zN*rAWC9P3EpTzNC}oOLNbC+*CjqY1Ca|Lv<~qq#8(hE zAWj4LG6W~xE45ePQb)F>M1V~P5$Xo&D%}5qrL%BvhPVb}PT+8mDo7DuRd__fBW^u? z55YPb`9$EL4={XvFmdHb=*QKQQce-Cj9y>cdj4A0CC&WQ{le>CzWr1v=S{=opXW~g zn)L|lH=nsWX|?3`!Jy~U+u7Nxd)ME*&@L#G7oHSy*ViIfzBBgZI4kze?X0`iSzqGd z=Nl8;HMrJXi+u*e^AK3nKZM)65br>697kAD=(b4CB7U*!bue7=~PYv?Cfi*pH;{v38gmNK8%7k!NRykByIv z4sSGc>HdME9Z2j; z4ZM(|jp4087E-;Ls#olS3vFA&Ki^*zY2UgNyuHa}lA7 zn3wxQi*&l#2Rv*tL(7u_(<^1ounrQSsDPTQBq5k zSTRqIEq>Pf>#u*(wFU zikhMJ;}^r+Dq%C|*m&X)u{50;cHP9no--oYBmEK1fkfN4k*8g!q@Q~`6tJ0DHi`a& z)KrIOPW1qOIa}XBmC)6SNuiFu3i6EC)A)&)XT5M%aKha6$L3K_n;6|6sTtj>g%3kM z;ehmB^pNmNH;UBGQL;$m1f@Xqqp_gLm@Pewq(CQ9LTrJV1$cvLwKmbp7V?f|yU$?& zwABHYt=f*&2^}T`vthPiGh5Xwrk1BQ;sbSHLA7aT7)hMmUY`RPGc~))>3qHglJK7m zf96#@$QG=!R`Fmp5U+$SFN3iFlOxes^io`m!4^7^81)w}1>}hQ2A0p`xR|-84aYZd zgS5RlA7=vtX;;IIZw6vgxRzOC0x6&*0Eer@KbXU8Kzt-F+bhq0s_3MEA8wONH`Ooecztz&^es7Y` z1z8Rh?MTk_pU ZlO`!$i6vS8mo|;HV<2L8o17+hy~#=M zz0r2~;&eU;GN@&YGP<%25fPt!*n^1dL0^2BPYQb)_~H~PLc5oVe&;4>)7F7LS~xxb ze!t)M`@WNWB$Jwp2uR2X^Sy!x7QDcBNcIssLXtg&5E5aaz)iNif)Tn8<2@TfElmi$ z2;*HC+c0DxFw$|U3mlWWb0Ec*uY<%xkU9r*8~_aT820P1Z-@CoRVpWh8(iUxgvTsD zeQv;@4Cc@n<~Rq(aW0Mu z;_VDY(VI@x4V6Zg7TL$!r^|) z#n(8%0@kn`=KG_4=h~4yHZatV_`b7IHbvtxHbu@x*c6ZS!-S$^?Z_F6hh2j=;Hodq zu0Yi)q_eS*APQJRfAqZ1hs^}`M7yzwBvbDT4ia870ggIQZl_!cbO{Lg7>yF z4q$FKV1#56Z*Pz`jaNi=a%g4X1EXU6vIWpz$0IJ&a zPTP%eSuL4VUL;Bhbh=L1&ED1-l_y-ulto(U`H+Waz+Z*SB&#Z#teytC-g#+2 ztG`wCQzF>BpHJ9zN=@Mu5^O#sIp_k_bMPe~Yt@5Rx{$3HrnPqh;6x2zn2K33e9!?M zaO$KB@S#w&vaV)HmH1o@m{UyRb0UF}U9~fSQC&6H7@eKBKoIuLV|(V}Bwa8HYPk-p zf|wIpEJ8^ECVIRb-kW~D15)rI-rHzgu!$aV9gEkopHCd??fxyiPdZ4hfwP5ybfk;> zzqEA-&T6E?6|jh#SX=@~LON*K{>`Qi!BJ%#s*Xk2!{Sp6kc9NCe(k@`?b~cDW!>lo z1_T&A(sKQuHSZfeT5XH^jZJ)v0hN&cvBUZ|pro}#>#t(*es2;tY;c=6+!LSKB>kT{ z_Wd7u&Uy-Iz(5A@(QNE>{yiYp92`8k#o2L)cU$2;A^i{fp13E89u#nt6&<>>87%6E L!ViZG{QmeG_=F;x literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q1/2.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q1/2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..93514c4dae1d49a91e61f8fed97b415f80cf6920 GIT binary patch literal 2180 zcmdT`UuauZ7(XY?ZEs_?q-*ZwhP8U@nvh}>lO`!$i6vS8mo|;HV<2L8o17+hy~#=M zz0r2~;&eU;GN@&YGP<%25fPt!*n^1dL0^2BPYQb)_~H~PLc5oVe&;4>)7F7LS~xxb ze!t)M`@WNWB$Jwp2uR2X^Sy!x7QDcBNcIssLXtg&5E5aaz)iNif)Tn8<2@TfElmi$ z2;*HC+c0DxFw$|U3mlWWb0Ec*uY<%xkU9r*8~_aT820P1Z-@CoRVpWh8(iUxgvTsD zeQv;@4Cc@n<~Rq(aW0Mu z;_VDY(VI@x4V6Zg7TL$!r^|) z#n(8%0@kn`=KG_4=h~4yHZatV_`b7IHbvtxHbu@x*c6ZS!-S$^?Z_F6hh2j=;Hodq zu0Yi)q_eS*APQJRfAqZ1hs^}`M7yzwBvbDT4ia870ggIQZl_!cbO{Lg7>yF z4q$FKV1#56Z*Pz`jaNi=a%g4X1EXU6vIWpz$0IJ&a zPTP%eSuL4VUL;Bhbh=L1&ED1-l_y-ulto(U`H+Waz+Z*SB&#Z#teytC-g#+2 ztG`wCQzF>BpHJ9zN=@Mu5^O#sIp_k_bMPe~Yt@5Rx{$3HrnPqh;6x2zn2K33e9!?M zaO$KB@S#w&vaV)HmH1o@m{UyRb0UF}U9~fSQC&6H7@eKBKoIuLV|(V}Bwa8HYPk-p zf|wIpEJ8^ECVIRb-kW~D15)rI-rHzgu!$aV9gEkopHCd??fxyiPdZ4hfwP5ybfk;> zzqEA-&T6E?6|jh#SX=@~LON*K{>`Qi!BJ%#s*Xk2!{Sp6kc9NCe(k@`?b~cDW!>lo z1_T&A(sKQuHSZfeT5XH^jZJ)v0hN&cvBUZ|pro}#>#t(*es2;tY;c=6+!LSKB>kT{ z_Wd7u&Uy-Iz(5A@(QNE>{yiYp92`8k#o2L)cU$2;A^i{fp13E89u#nt6&<>>87%6E L!ViZG{QmeG_=F;x literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q2/1.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q2/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e8ae33ef24f6c12844a63cedf27321a81b863866 GIT binary patch literal 2110 zcmcguUrbwN6#u@`%Wg{%=6Y{#w@&YzbS!Dn0)tgLZP^%q2X$j}W}4>Gel6GD_AB?# z0F5Rt5=|t_Vhko^K{6AJ8R21Z(WgCVe8XpaF=p!95?r#l_@IgByKUhD?p2c0@7!~L zzjJ=)eC0qYc|Iy2Atj851wS~vzy~EaaUmqRZy|(4Fw}aJxj%s22Rn-q>Ib_8h5(4c zj)5`K-c(<5EY)>@R#P#Tr5FbL3n&b5w+^9c@Y7(|z@LN|q{<3$hbo+s@R*&Xy_uCB z!yHNp+)GV}3$%_^A}kEVWh^XW2bG_D@anhE{d^!P@jtGwuaA6pJ7FhE#cPjKr@R~Q zJaQ#5Ka&5fAz8!Uf7`$I;m5xua5KM|_>F4~A2REc-~ZYA#-W}^37iP8l0OpOjW5dY ze()f%xA(&5L(uzC_^kHc3od)S0QM$W1nfB2C|D2JWiUp9RKH^3Q)dg@iO9&|n>qXeS)*6Yae*Fxgj|Wk$qN% zh|nTta@Rb<{5}Wp;7nO3h9N7N47Ez8Ax}_hnDUgRW~5wGt~qdKECWgoK^B8*SBb5(z^YDA%e2dptE41P z=!%+!4BN6AM^bW@l&PUYbp?vf*e_4v(O{b`agqe)o9#K^RsXb7fZUBL%I&`+8D)}D z6%8L7ABMVnr*e3xbLTWT%d-%6&Vjp!=TPU7=TV+(io+B^a84~LnhaCdaoXd*Y^x?l zMky;!6bNDwUj^n75>NbQS_N0jFo1orsCWDv|{XkEL>WC1u#0v zTbiQFWhGNk@&x)Xt0j}l6GSN)(C%AQBXZyhujCbc4PQYrzKd7TOv_J4dX7&t+gHyO zeI1bL^=EmdNF$!;H>fT4ez?O~XBz z_OlNFja7hQDyC($z?$+x)JN0s1yZ!Ku4YJu*iZ##6_d31kif`I)fm92u9~xq&f@!^ z2>V@Q&rHQrG;K_($0j_+;zJCO zxb&pW99rPEWI9Lb5W2vD&|NDwv|7IK&K(I|fu-`p<Pa+|Z(AkKT>eO&rqcb&Kr Z5dEmtTUK<~p=PA0qZs^+PQiZ;{{kE(4l@7% literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q2/2.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q2/2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e8ae33ef24f6c12844a63cedf27321a81b863866 GIT binary patch literal 2110 zcmcguUrbwN6#u@`%Wg{%=6Y{#w@&YzbS!Dn0)tgLZP^%q2X$j}W}4>Gel6GD_AB?# z0F5Rt5=|t_Vhko^K{6AJ8R21Z(WgCVe8XpaF=p!95?r#l_@IgByKUhD?p2c0@7!~L zzjJ=)eC0qYc|Iy2Atj851wS~vzy~EaaUmqRZy|(4Fw}aJxj%s22Rn-q>Ib_8h5(4c zj)5`K-c(<5EY)>@R#P#Tr5FbL3n&b5w+^9c@Y7(|z@LN|q{<3$hbo+s@R*&Xy_uCB z!yHNp+)GV}3$%_^A}kEVWh^XW2bG_D@anhE{d^!P@jtGwuaA6pJ7FhE#cPjKr@R~Q zJaQ#5Ka&5fAz8!Uf7`$I;m5xua5KM|_>F4~A2REc-~ZYA#-W}^37iP8l0OpOjW5dY ze()f%xA(&5L(uzC_^kHc3od)S0QM$W1nfB2C|D2JWiUp9RKH^3Q)dg@iO9&|n>qXeS)*6Yae*Fxgj|Wk$qN% zh|nTta@Rb<{5}Wp;7nO3h9N7N47Ez8Ax}_hnDUgRW~5wGt~qdKECWgoK^B8*SBb5(z^YDA%e2dptE41P z=!%+!4BN6AM^bW@l&PUYbp?vf*e_4v(O{b`agqe)o9#K^RsXb7fZUBL%I&`+8D)}D z6%8L7ABMVnr*e3xbLTWT%d-%6&Vjp!=TPU7=TV+(io+B^a84~LnhaCdaoXd*Y^x?l zMky;!6bNDwUj^n75>NbQS_N0jFo1orsCWDv|{XkEL>WC1u#0v zTbiQFWhGNk@&x)Xt0j}l6GSN)(C%AQBXZyhujCbc4PQYrzKd7TOv_J4dX7&t+gHyO zeI1bL^=EmdNF$!;H>fT4ez?O~XBz z_OlNFja7hQDyC($z?$+x)JN0s1yZ!Ku4YJu*iZ##6_d31kif`I)fm92u9~xq&f@!^ z2>V@Q&rHQrG;K_($0j_+;zJCO zxb&pW99rPEWI9Lb5W2vD&|NDwv|7IK&K(I|fu-`p<Pa+|Z(AkKT>eO&rqcb&Kr Z5dEmtTUK<~p=PA0qZs^+PQiZ;{{kE(4l@7% literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q3/1.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q3/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..aae46dd28284690abb2f766eadb0d653931f22af GIT binary patch literal 1902 zcmb_dOKcle6n)R1WMW*BfZu2=obn{%k*Vxpr;h7NZe`PiI8mCoA(6TWiYE5h88Y#> z^AT~C6>0$q2}BoBRaFHQgoLPAR9R$0L<2hzi*AtEpsHmPLdb>)iKybfv7Js56*gG9 zzBBKhbM9xx*M;0#yQ5BvL@PsYW z-vt;nao5X0pI~wuJ7C=8+VQI|Vt-EH&iH=%^!m=jtW7G+>_%oj9LsXKtqW_%|DJH? zz8ZdTc`xD4Vd00-UpKS)2bTll%RAXJe8#`q%(iq|Dp!BIu(tBYeq=Zw^hG(iIT<@YG&nMSKF9}sn1)*LsvmU^ z<~sp!ND-3mxyWY+lAcp#ec`3J{Yej`gihx{ynn#Xy^bSHt{L)TE|o}~eI9YiWSqHy z)2EnAClky8iMjrG5}YTMcy*Qm(`Q8hrcc_yfs{Q)zjKVB4`XECjWHs01G6#RZX-J$ z2QXBpQn9G%C8b)Srbf+qnU+PE`P{ z*8AusODWoEK63Ajr6H%{2-7)DEk#>G;4fLSVHyqA0%WnOmSIUMEy=ZB;HokUD!Nux zic#;GDJmKD#hR&^kaj_-mQ?u!pt>isM{4R>?vT^3K(LRNEnU$pL#=n8rmQEeq^SzN z*d;+@OTkt80-%bzW*9hpkrpLgE?X*;QdX)^h)f`A_IyvaP!A5`FHj%Z=>8>AlZp#c zB?PgGce2NbIRVu37vxcPLHSc?|>19Uxw?PwE@J`;N zYnn-(L&OzEf=`=;hXOB?TyGq7XtPDjFztIk0FE^QhAEkr5r*Ho4^I8sEKEvLYr0aD z8^mXtz>;LjVLu{_+}S(>n5HXcozdBOJ0xLmFuP|}Jgdzb^GdA+Yk*iK+Tx;IH37y4 zf}!9|kqbc!VZ;TUg%cMa;ICnR1&dtf$U+ov;eFDR<{X?`7)Xa!pSbD@5q~4o!3o&J z8<-yjNCuW%YtM;~f#GW~4ot@;>|*{e43LcQw3R3BdiKk7jU|f#i2;=n{$o>#+wJ>Gt> z^AT~C6>0$q2}BoBRaFHQgoLPAR9R$0L<2hzi*AtEpsHmPLdb>)iKybfv7Js56*gG9 zzBBKhbM9xx*M;0#yQ5BvL@PsYW z-vt;nao5X0pI~wuJ7C=8+VQI|Vt-EH&iH=%^!m=jtW7G+>_%oj9LsXKtqW_%|DJH? zz8ZdTc`xD4Vd00-UpKS)2bTll%RAXJe8#`q%(iq|Dp!BIu(tBYeq=Zw^hG(iIT<@YG&nMSKF9}sn1)*LsvmU^ z<~sp!ND-3mxyWY+lAcp#ec`3J{Yej`gihx{ynn#Xy^bSHt{L)TE|o}~eI9YiWSqHy z)2EnAClky8iMjrG5}YTMcy*Qm(`Q8hrcc_yfs{Q)zjKVB4`XECjWHs01G6#RZX-J$ z2QXBpQn9G%C8b)Srbf+qnU+PE`P{ z*8AusODWoEK63Ajr6H%{2-7)DEk#>G;4fLSVHyqA0%WnOmSIUMEy=ZB;HokUD!Nux zic#;GDJmKD#hR&^kaj_-mQ?u!pt>isM{4R>?vT^3K(LRNEnU$pL#=n8rmQEeq^SzN z*d;+@OTkt80-%bzW*9hpkrpLgE?X*;QdX)^h)f`A_IyvaP!A5`FHj%Z=>8>AlZp#c zB?PgGce2NbIRVu37vxcPLHSc?|>19Uxw?PwE@J`;N zYnn-(L&OzEf=`=;hXOB?TyGq7XtPDjFztIk0FE^QhAEkr5r*Ho4^I8sEKEvLYr0aD z8^mXtz>;LjVLu{_+}S(>n5HXcozdBOJ0xLmFuP|}Jgdzb^GdA+Yk*iK+Tx;IH37y4 zf}!9|kqbc!VZ;TUg%cMa;ICnR1&dtf$U+ov;eFDR<{X?`7)Xa!pSbD@5q~4o!3o&J z8<-yjNCuW%YtM;~f#GW~4ot@;>|*{e43LcQw3R3BdiKk7jU|f#i2;=n{$o>#+wJ>Gt>7%Q6rOR{$=Y>rTD&V;xO9^?#xzbECv_XQZskz&XH|(?f+#`|Hul(Fvf0`0 z4~YX3LKMP@0zwt4st`dXIG`SYLrxq3MF=GqE`>`)1V|ADR7Lyuh-l= z3m{EEV5FU{o^&%;dB9g(F~b`8dl}?ups=kDaPit6gjPULf$e9|;3_W!YO=z^K|Jg@ z>1>33H2jI1f#$F%dH)q02X@``Z|MmR*g9k3&NqcX7pkMi- zzUKu%%CG#Ce)Z1z`Mzdf48j72 zmA1}hpwip#WvZ33+8(Q|dM(b+aI z*gvqjf|zSA;r-f}{-i21;RH{J-r(m{b>Gn}naY7@=tV)b3k3ywr4!A4rQ0BvY+|yw z+N7f851f?bG5S17Lssz&GEY8*;{L=WEZZMG9sjiP2-xMmFHxXGbK|7k6!n#fVPe{7 zTas16Uxh6qt5&HT%lrlyxmZ-m;C{$*AvwXxW1jvfkxZTB6j_Z)f-Cg#0B2BY3Hy=Y zYC%CS9i>JdZZ9dX8$5Ir(zcvu9{?IE0Mn8!+iZd9;svXZj#|``Rozf>WSzLT0?b2O zTYN}hWPim5Fls1PnbDcO9)hq}hCQ<^?xUk-NztpYbr8#hPE5cF0!$2q!lCOCJ`5$a zAU;%^xZn~4;@eofiX(h##|Vzh;tkSfaW$M-45V$tHu9M(Ecn(VZS8-m;tZijYqN^Mvx@}fQUa#9obbGfQ>TQ>} zf&rBZ-g7cW?$&SctEb}de%~X$JuBHseB s((274cQ`+I#j8B*r-J{>z!EQq!~lwVb=iP7(rTM9P!oJgv+!a657SER*#H0l literal 0 HcmV?d00001 diff --git a/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q4/2.parquet b/exec/java-exec/src/test/resources/multilevel/parquet2/1995/Q4/2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..bae64e30d639ca505d6035301f48d91a8aab2ee5 GIT binary patch literal 2013 zcmcIlO>7%Q6rOR{$=Y>rTD&V;xO9^?#xzbECv_XQZskz&XH|(?f+#`|Hul(Fvf0`0 z4~YX3LKMP@0zwt4st`dXIG`SYLrxq3MF=GqE`>`)1V|ADR7Lyuh-l= z3m{EEV5FU{o^&%;dB9g(F~b`8dl}?ups=kDaPit6gjPULf$e9|;3_W!YO=z^K|Jg@ z>1>33H2jI1f#$F%dH)q02X@``Z|MmR*g9k3&NqcX7pkMi- zzUKu%%CG#Ce)Z1z`Mzdf48j72 zmA1}hpwip#WvZ33+8(Q|dM(b+aI z*gvqjf|zSA;r-f}{-i21;RH{J-r(m{b>Gn}naY7@=tV)b3k3ywr4!A4rQ0BvY+|yw z+N7f851f?bG5S17Lssz&GEY8*;{L=WEZZMG9sjiP2-xMmFHxXGbK|7k6!n#fVPe{7 zTas16Uxh6qt5&HT%lrlyxmZ-m;C{$*AvwXxW1jvfkxZTB6j_Z)f-Cg#0B2BY3Hy=Y zYC%CS9i>JdZZ9dX8$5Ir(zcvu9{?IE0Mn8!+iZd9;svXZj#|``Rozf>WSzLT0?b2O zTYN}hWPim5Fls1PnbDcO9)hq}hCQ<^?xUk-NztpYbr8#hPE5cF0!$2q!lCOCJ`5$a zAU;%^xZn~4;@eofiX(h##|Vzh;tkSfaW$M-45V$tHu9M(Ecn(VZS8-m;tZijYqN^Mvx@}fQUa#9obbGfQ>TQ>} zf&rBZ-g7cW?$&SctEb}de%~X$JuBHseB s((274cQ`+I#j8B*r-J{>z!EQq!~lwVb=iP7(rTM9P!oJgv+!a657SER*#H0l literal 0 HcmV?d00001 From ef37b7743a1ebadc4dbe8c4b26787cc0715705f6 Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Tue, 12 Jul 2016 18:01:24 -0700 Subject: [PATCH 5/7] Populate only fileSet when expanding using the metadata cache. --- .../exec/store/parquet/ParquetGroupScan.java | 33 ++++++++----------- 1 file changed, 14 insertions(+), 19 deletions(-) diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java index 4973beac581..df08a1bc546 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java @@ -571,22 +571,24 @@ public long getRowCount() { // get (and set internal field) the metadata for the directory by reading the metadata file this.parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath.toString()); - List fileNames = Lists.newArrayList(); List fileStatuses = selection.getStatuses(fs); + if (fileSet == null) { + fileSet = Sets.newHashSet(); + } + final Path first = fileStatuses.get(0).getPath(); if (fileStatuses.size() == 1 && selection.getSelectionRoot().equals(first.toString())) { // we are selecting all files from selection root. Expand the file list from the cache for (Metadata.ParquetFileMetadata file : parquetTableMetadata.getFiles()) { - fileNames.add(file.getPath()); + fileSet.add(file.getPath()); } - // we don't need to populate fileSet as all files are selected + } else if (selection.isExpandedPartial() && cacheFileRoot != null) { this.parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath.toString()); for (Metadata.ParquetFileMetadata file : this.parquetTableMetadata.getFiles()) { - fileNames.add(file.getPath()); + fileSet.add(file.getPath()); } - fileSet = Sets.newHashSet(fileNames); } else { // we need to expand the files from fileStatuses for (FileStatus status : fileStatuses) { @@ -595,30 +597,23 @@ public long getRowCount() { final Path metaPath = new Path(status.getPath(), Metadata.METADATA_FILENAME); final Metadata.ParquetTableMetadataBase metadata = Metadata.readBlockMeta(fs, metaPath.toString()); for (Metadata.ParquetFileMetadata file : metadata.getFiles()) { - fileNames.add(file.getPath()); + // fileNames.add(file.getPath()); + fileSet.add(file.getPath()); } } else { final Path path = Path.getPathWithoutSchemeAndAuthority(status.getPath()); - fileNames.add(path.toString()); + fileSet.add(path.toString()); } } - - // populate fileSet so we only keep the selected row groups - fileSet = Sets.newHashSet(fileNames); - } - - List finalFileNames; - if (fileSet != null) { - finalFileNames = Lists.newArrayList(fileSet); - } else { - finalFileNames = fileNames; } - if (finalFileNames.isEmpty()) { + if (fileSet.isEmpty()) { // no files were found, most likely we tried to query some empty sub folders throw UserException.validationError().message("The table you tried to query is empty").build(logger); } + List fileNames = Lists.newArrayList(fileSet); + // when creating the file selection, set the selection root in the form /a/b instead of // file:/a/b. The reason is that the file names above have been created in the form // /a/b/c.parquet and the format of the selection root must match that of the file names @@ -630,7 +625,7 @@ public long getRowCount() { // because create() changes the root to include the scheme and authority; In future, if create() // is the preferred way to instantiate a file selection, we may need to do something different... // WARNING: file statuses and file names are inconsistent - FileSelection newSelection = new FileSelection(selection.getStatuses(fs), finalFileNames, metaRootPath.toString(), cacheFileRoot); + FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString(), cacheFileRoot); newSelection.setExpandedFully(); return newSelection; From 7ae8f7a16a6a4e2821e122c7850711ff5de8d086 Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Tue, 12 Jul 2016 22:10:07 -0700 Subject: [PATCH 6/7] Remove cacheFileRoot parameter from FileGroupScan's clone() method and instead leverage it from FileSelection. --- .../exec/physical/base/AbstractFileGroupScan.java | 2 +- .../drill/exec/physical/base/FileGroupScan.java | 2 +- .../exec/planner/FileSystemPartitionDescriptor.java | 3 +-- .../exec/planner/ParquetPartitionDescriptor.java | 4 ++-- .../apache/drill/exec/store/dfs/FileSelection.java | 11 ++++++++--- .../drill/exec/store/dfs/easy/EasyGroupScan.java | 2 +- .../drill/exec/store/parquet/ParquetGroupScan.java | 8 ++++---- 7 files changed, 18 insertions(+), 14 deletions(-) diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java index 89ceecfa469..606aa4da77b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractFileGroupScan.java @@ -38,7 +38,7 @@ public void modifyFileSelection(FileSelection selection) { } @Override - public FileGroupScan clone(FileSelection selection, String cacheFileRoot) throws IOException { + public FileGroupScan clone(FileSelection selection) throws IOException { throw new UnsupportedOperationException(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java index 66e8375a257..9d4767edae9 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileGroupScan.java @@ -25,6 +25,6 @@ public interface FileGroupScan extends GroupScan { public void modifyFileSelection(FileSelection selection); - public FileGroupScan clone(FileSelection selection, String cacheFileRoot) throws IOException; + public FileGroupScan clone(FileSelection selection) throws IOException; } \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java index 46738d146db..10e11bf8a10 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java @@ -223,12 +223,11 @@ public TableScan createTableScan(List newPartitionLocation, S } if (scanRel instanceof DrillScanRel) { -// final FormatSelection formatSelection = (FormatSelection)((DynamicDrillTable)scanRel.getTable()).getSelection(); final FormatSelection formatSelection = (FormatSelection)table.getSelection(); final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(), cacheFileRoot, formatSelection.getSelection().getDirStatus()); final FileGroupScan newGroupScan = - ((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection, cacheFileRoot); + ((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), scanRel.getTable(), diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java index 7ed36af3147..8b311ff904d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java @@ -81,8 +81,8 @@ public int getMaxHierarchyLevel() { } private GroupScan createNewGroupScan(List newFiles, String cacheFileRoot) throws IOException { - final FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation()); - final FileGroupScan newScan = ((FileGroupScan)scanRel.getGroupScan()).clone(newSelection, cacheFileRoot); + final FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation(), cacheFileRoot); + final FileGroupScan newScan = ((FileGroupScan)scanRel.getGroupScan()).clone(newSelection); return newScan; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java index 8c7c0882bc6..219948f555f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java @@ -281,13 +281,14 @@ public static FileSelection create(final DrillFileSystem fs, final String parent * @param statuses list of file statuses * @param files list of files * @param root root path for selections - * + * @param cacheFileRoot root path for metadata cache (null for no metadata cache) * @return null if creation of {@link FileSelection} fails with an {@link IllegalArgumentException} * otherwise a new selection. * * @see FileSelection#FileSelection(List, List, String) */ - public static FileSelection create(final List statuses, final List files, final String root) { + public static FileSelection create(final List statuses, final List files, final String root, + final String cacheFileRoot) { final boolean bothNonEmptySelection = (statuses != null && statuses.size() > 0) && (files != null && files.size() > 0); final boolean bothEmptySelection = (statuses == null || statuses.size() == 0) && (files == null || files.size() == 0); @@ -307,7 +308,11 @@ public static FileSelection create(final List statuses, final List statuses, final List files, final String root) { + return FileSelection.create(statuses, files, root, null); } public static FileSelection createFromDirectories(final List dirPaths, final FileSelection selection) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java index 409f80d56d2..7a80db3199b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java @@ -265,7 +265,7 @@ public GroupScan clone(List columns) { } @Override - public FileGroupScan clone(FileSelection selection, String cacheFileRoot /* ignored */) throws IOException { + public FileGroupScan clone(FileSelection selection) throws IOException { EasyGroupScan newScan = new EasyGroupScan(this); newScan.initFromSelection(selection, formatPlugin); newScan.mappings = null; /* the mapping will be created later when we get specific scan diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java index df08a1bc546..8029911d4ba 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java @@ -871,10 +871,10 @@ public GroupScan clone(List columns) { } @Override - public FileGroupScan clone(FileSelection selection, String cacheFileRoot) throws IOException { + public FileGroupScan clone(FileSelection selection) throws IOException { ParquetGroupScan newScan = new ParquetGroupScan(this); newScan.modifyFileSelection(selection); - newScan.cacheFileRoot = cacheFileRoot; + newScan.cacheFileRoot = selection.cacheFileRoot; newScan.init(); return newScan; } @@ -913,9 +913,9 @@ public GroupScan applyLimit(long maxRecords) { } try { - FileSelection newSelection = new FileSelection(null, Lists.newArrayList(fileNames), getSelectionRoot()); + FileSelection newSelection = new FileSelection(null, Lists.newArrayList(fileNames), getSelectionRoot(), cacheFileRoot); logger.debug("applyLimit() reduce parquet file # from {} to {}", fileSet.size(), fileNames.size()); - return this.clone(newSelection, this.cacheFileRoot); + return this.clone(newSelection); } catch (IOException e) { logger.warn("Could not apply rowcount based prune due to Exception : {}", e); return null; From ce509af51146e99cf8a696591860d62a4eabd358 Mon Sep 17 00:00:00 2001 From: Aman Sinha Date: Fri, 15 Jul 2016 12:08:42 -0700 Subject: [PATCH 7/7] Keep track of whether all partitions were previously pruned and process this state where needed. --- .../planner/sql/HivePartitionDescriptor.java | 2 +- .../planner/AbstractPartitionDescriptor.java | 3 +- .../exec/planner/DFSDirPartitionLocation.java | 12 +++++++ .../FileSystemPartitionDescriptor.java | 18 ++++++---- .../planner/ParquetPartitionDescriptor.java | 16 +++++---- .../exec/planner/PartitionDescriptor.java | 8 +++-- .../drill/exec/planner/PartitionLocation.java | 5 +++ .../exec/planner/SimplePartitionLocation.java | 5 +++ .../logical/partition/PruneScanRule.java | 22 +++++++----- .../drill/exec/store/dfs/FileSelection.java | 34 +++++++++++++----- .../exec/store/parquet/ParquetGroupScan.java | 35 ++++++++++--------- .../parquet/TestParquetMetadataCache.java | 32 ++++++++++++++--- 12 files changed, 137 insertions(+), 55 deletions(-) diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java index 8a7e43a7eeb..d42aea7868f 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java @@ -152,7 +152,7 @@ protected void createPartitionSublists() { } @Override - public TableScan createTableScan(List newPartitions) throws Exception { + public TableScan createTableScan(List newPartitions, boolean wasAllPartitionsPruned /* ignored */) throws Exception { GroupScan newGroupScan = createNewGroupScan(newPartitions); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java index 49b00c4dc13..98794926c6c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractPartitionDescriptor.java @@ -64,7 +64,8 @@ public boolean supportsSinglePartOptimization() { @Override - public TableScan createTableScan(List newPartitions, String cacheFileRoot) throws Exception { + public TableScan createTableScan(List newPartitions, String cacheFileRoot, + boolean isAllPruned) throws Exception { throw new UnsupportedOperationException(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java index da3aa681e82..a4d2b8163c1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java @@ -67,4 +67,16 @@ public boolean isCompositePartition() { return true; } + @Override + public String getCompositePartitionPath() { + String path = ""; + for (int i=0; i < dirs.length; i++) { + if (dirs[i] == null) { // get the prefix + break; + } + path += "/" + dirs[i]; + } + return path; + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java index 10e11bf8a10..ba18bbe2220 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java @@ -209,7 +209,8 @@ protected Pair, Boolean> getFileLocationsAndStatus() { } @Override - public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot) throws Exception { + public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot, + boolean wasAllPartitionsPruned) throws Exception { List newFiles = Lists.newArrayList(); for (final PartitionLocation location : newPartitionLocation) { if (!location.isCompositePartition()) { @@ -225,7 +226,7 @@ public TableScan createTableScan(List newPartitionLocation, S if (scanRel instanceof DrillScanRel) { final FormatSelection formatSelection = (FormatSelection)table.getSelection(); final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(), - cacheFileRoot, formatSelection.getSelection().getDirStatus()); + cacheFileRoot, wasAllPartitionsPruned, formatSelection.getSelection().getDirStatus()); final FileGroupScan newGroupScan = ((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection); return new DrillScanRel(scanRel.getCluster(), @@ -236,17 +237,19 @@ public TableScan createTableScan(List newPartitionLocation, S ((DrillScanRel) scanRel).getColumns(), true /*filter pushdown*/); } else if (scanRel instanceof EnumerableTableScan) { - return createNewTableScanFromSelection((EnumerableTableScan)scanRel, newFiles, cacheFileRoot); + return createNewTableScanFromSelection((EnumerableTableScan)scanRel, newFiles, cacheFileRoot, + wasAllPartitionsPruned); } else { throw new UnsupportedOperationException("Only DrillScanRel and EnumerableTableScan is allowed!"); } } - private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, List newFiles, String cacheFileRoot) { + private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, List newFiles, String cacheFileRoot, + boolean wasAllPartitionsPruned) { final RelOptTableImpl t = (RelOptTableImpl) oldScan.getTable(); final FormatSelection formatSelection = (FormatSelection) table.getSelection(); final FileSelection newFileSelection = new FileSelection(null, newFiles, getBaseTableLocation(), - cacheFileRoot, formatSelection.getSelection().getDirStatus()); + cacheFileRoot, wasAllPartitionsPruned, formatSelection.getSelection().getDirStatus()); final FormatSelection newFormatSelection = new FormatSelection(formatSelection.getFormat(), newFileSelection); final DrillTranslatableTable newTable = new DrillTranslatableTable( new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(), @@ -258,8 +261,9 @@ private TableScan createNewTableScanFromSelection(EnumerableTableScan oldScan, L } @Override - public TableScan createTableScan(List newPartitionLocation) throws Exception { - return createTableScan(newPartitionLocation, null); + public TableScan createTableScan(List newPartitionLocation, + boolean wasAllPartitionsPruned) throws Exception { + return createTableScan(newPartitionLocation, null, wasAllPartitionsPruned); } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java index 8b311ff904d..2c8ca9598d1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java @@ -80,8 +80,10 @@ public int getMaxHierarchyLevel() { return partitionColumns.size(); } - private GroupScan createNewGroupScan(List newFiles, String cacheFileRoot) throws IOException { - final FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation(), cacheFileRoot); + private GroupScan createNewGroupScan(List newFiles, String cacheFileRoot, + boolean wasAllPartitionsPruned) throws IOException { + final FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation(), + cacheFileRoot, wasAllPartitionsPruned); final FileGroupScan newScan = ((FileGroupScan)scanRel.getGroupScan()).clone(newSelection); return newScan; } @@ -131,13 +133,14 @@ protected void createPartitionSublists() { } @Override - public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot) throws Exception { + public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot, + boolean wasAllPartitionsPruned) throws Exception { List newFiles = Lists.newArrayList(); for (final PartitionLocation location : newPartitionLocation) { newFiles.add(location.getEntirePartitionLocation()); } - final GroupScan newGroupScan = createNewGroupScan(newFiles, cacheFileRoot); + final GroupScan newGroupScan = createNewGroupScan(newFiles, cacheFileRoot, wasAllPartitionsPruned); return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), @@ -149,8 +152,9 @@ public TableScan createTableScan(List newPartitionLocation, S } @Override - public TableScan createTableScan(List newPartitionLocation) throws Exception { - return createTableScan(newPartitionLocation, null); + public TableScan createTableScan(List newPartitionLocation, + boolean wasAllPartitionsPruned) throws Exception { + return createTableScan(newPartitionLocation, null, wasAllPartitionsPruned); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java index 50c160bcd47..4d1bfddb191 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionDescriptor.java @@ -77,20 +77,24 @@ void populatePartitionVectors(ValueVector[] vectors, List par /** * Methods create a new TableScan rel node, given the lists of new partitions or new files to SCAN. * @param newPartitions + * @param wasAllPartitionsPruned * @return * @throws Exception */ - public TableScan createTableScan(List newPartitions) throws Exception; + public TableScan createTableScan(List newPartitions, + boolean wasAllPartitionsPruned) throws Exception; /** * Create a new TableScan rel node, given the lists of new partitions or new files to scan and a path * to a metadata cache file * @param newPartitions * @param cacheFileRoot + * @param wasAllPartitionsPruned * @return * @throws Exception */ - public TableScan createTableScan(List newPartitions, String cacheFileRoot) throws Exception; + public TableScan createTableScan(List newPartitions, String cacheFileRoot, + boolean wasAllPartitionsPruned) throws Exception; public boolean supportsSinglePartOptimization(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java index 31c926975c7..b6396b2bc75 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PartitionLocation.java @@ -55,4 +55,9 @@ public interface PartitionLocation { */ public boolean isCompositePartition(); + /** + * Returns the path string of directory names only for composite partition + */ + public String getCompositePartitionPath(); + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java index 523169e7e08..7c4c22f064f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimplePartitionLocation.java @@ -33,6 +33,11 @@ public boolean isCompositePartition() { return false; } + @Override + public String getCompositePartitionPath() { + throw new UnsupportedOperationException(); + } + @Override public List getPartitionLocationRecursive() { return ImmutableList.of(this); 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 8a4879036c0..209e03da547 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 @@ -48,6 +48,7 @@ import org.apache.drill.exec.planner.FileSystemPartitionDescriptor; import org.apache.drill.exec.planner.PartitionDescriptor; import org.apache.drill.exec.planner.PartitionLocation; +import org.apache.drill.exec.planner.SimplePartitionLocation; import org.apache.drill.exec.planner.logical.DrillOptiq; import org.apache.drill.exec.planner.logical.DrillParseContext; import org.apache.drill.exec.planner.logical.DrillScanRel; @@ -79,7 +80,6 @@ public abstract class PruneScanRule extends StoragePluginOptimizerRule { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PruneScanRule.class); final OptimizerRulesContext optimizerContext; - boolean wasAllPartitionsPruned = false; // whether all partitions were previously eliminated public PruneScanRule(RelOptRuleOperand operand, String id, OptimizerRulesContext optimizerContext) { super(operand, id); @@ -144,10 +144,6 @@ public static final RelOptRule getDirFilterOnScan(OptimizerRulesContext optimize } protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectRel, TableScan scanRel) { - if (wasAllPartitionsPruned) { - // if previously we had already pruned out all the partitions, we should exit early - return; - } final String pruningClassName = getClass().getName(); logger.info("Beginning partition pruning, pruning class: {}", pruningClassName); @@ -359,6 +355,8 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR // handle the case all partitions are filtered out. boolean canDropFilter = true; + boolean wasAllPartitionsPruned = false; + String cacheFileRoot = null; if (newPartitions.isEmpty()) { assert firstLocation != null; @@ -366,8 +364,16 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR // In such case, we should not drop filter. newPartitions.add(firstLocation.getPartitionLocationRecursive().get(0)); canDropFilter = false; + // NOTE: with DRILL-4530, the PruneScanRule may be called with only a list of + // directories first and the non-composite partition location will still return + // directories, not files. So, additional processing is done depending on this flag wasAllPartitionsPruned = true; logger.info("All {} partitions were pruned; added back a single partition to allow creating a schema", numTotal); + + // set the cacheFileRoot appropriately + if (firstLocation.isCompositePartition()) { + cacheFileRoot = descriptor.getBaseTableLocation() + firstLocation.getCompositePartitionPath(); + } } logger.info("Pruned {} partitions down to {}", numTotal, newPartitions.size()); @@ -382,8 +388,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR condition = condition.accept(reverseVisitor); pruneCondition = pruneCondition.accept(reverseVisitor); - String cacheFileRoot = null; - if (checkForSingle && isSinglePartition) { + if (checkForSingle && isSinglePartition && !wasAllPartitionsPruned) { // if metadata cache file could potentially be used, then assign a proper cacheFileRoot String path = ""; for (int j = 0; j <= maxIndex; j++) { @@ -393,7 +398,8 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR } RelNode inputRel = descriptor.supportsSinglePartOptimization() ? - descriptor.createTableScan(newPartitions, cacheFileRoot) : descriptor.createTableScan(newPartitions); + descriptor.createTableScan(newPartitions, cacheFileRoot, wasAllPartitionsPruned) : + descriptor.createTableScan(newPartitions, wasAllPartitionsPruned); if (projectRel != null) { inputRel = projectRel.copy(projectRel.getTraitSet(), Collections.singletonList(inputRel)); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java index 219948f555f..d357c39e269 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java @@ -65,7 +65,10 @@ private enum StatusType { } private StatusType dirStatus; - private boolean hadWildcard = false; // whether this selection previously had a wildcard + // whether this selection previously had a wildcard + private boolean hadWildcard = false; + // whether all partitions were previously pruned for this selection + private boolean wasAllPartitionsPruned = false; /** * Creates a {@link FileSelection selection} out of given file statuses/files and selection root. @@ -75,20 +78,22 @@ private enum StatusType { * @param selectionRoot root path for selections */ public FileSelection(final List statuses, final List files, final String selectionRoot) { - this(statuses, files, selectionRoot, null, StatusType.NOT_CHECKED); + this(statuses, files, selectionRoot, null, false, StatusType.NOT_CHECKED); } - public FileSelection(final List statuses, final List files, final String selectionRoot, final String cacheFileRoot) { - this(statuses, files, selectionRoot, cacheFileRoot, StatusType.NOT_CHECKED); + public FileSelection(final List statuses, final List files, final String selectionRoot, + final String cacheFileRoot, final boolean wasAllPartitionsPruned) { + this(statuses, files, selectionRoot, cacheFileRoot, wasAllPartitionsPruned, StatusType.NOT_CHECKED); } public FileSelection(final List statuses, final List files, final String selectionRoot, - final String cacheFileRoot, final StatusType dirStatus) { + final String cacheFileRoot, final boolean wasAllPartitionsPruned, final StatusType dirStatus) { this.statuses = statuses; this.files = files; this.selectionRoot = Preconditions.checkNotNull(selectionRoot); this.dirStatus = dirStatus; this.cacheFileRoot = cacheFileRoot; + this.wasAllPartitionsPruned = wasAllPartitionsPruned; } /** @@ -101,6 +106,8 @@ protected FileSelection(final FileSelection selection) { this.selectionRoot = selection.selectionRoot; this.dirStatus = selection.dirStatus; this.cacheFileRoot = selection.cacheFileRoot; + this.hadWildcard = selection.hadWildcard; + this.wasAllPartitionsPruned = selection.wasAllPartitionsPruned; } public String getSelectionRoot() { @@ -202,6 +209,10 @@ public StatusType getDirStatus() { return dirStatus; } + public boolean wasAllPartitionsPruned() { + return this.wasAllPartitionsPruned; + } + private static String commonPath(final List statuses) { if (statuses == null || statuses.isEmpty()) { return ""; @@ -288,7 +299,7 @@ public static FileSelection create(final DrillFileSystem fs, final String parent * @see FileSelection#FileSelection(List, List, String) */ public static FileSelection create(final List statuses, final List files, final String root, - final String cacheFileRoot) { + final String cacheFileRoot, final boolean wasAllPartitionsPruned) { final boolean bothNonEmptySelection = (statuses != null && statuses.size() > 0) && (files != null && files.size() > 0); final boolean bothEmptySelection = (statuses == null || statuses.size() == 0) && (files == null || files.size() == 0); @@ -308,11 +319,11 @@ public static FileSelection create(final List statuses, final List statuses, final List files, final String root) { - return FileSelection.create(statuses, files, root, null); + return FileSelection.create(statuses, files, root, null, false); } public static FileSelection createFromDirectories(final List dirPaths, final FileSelection selection) { @@ -368,7 +379,12 @@ public List getFileStatuses() { } public boolean supportDirPrunig() { - return isExpandedFully() || isExpandedPartial(); + if (isExpandedFully() || isExpandedPartial()) { + if (!wasAllPartitionsPruned) { + return true; + } + } + return false; } public void setHadWildcard(boolean wc) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java index 8029911d4ba..b83847225cb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java @@ -586,8 +586,13 @@ public long getRowCount() { } else if (selection.isExpandedPartial() && cacheFileRoot != null) { this.parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath.toString()); - for (Metadata.ParquetFileMetadata file : this.parquetTableMetadata.getFiles()) { - fileSet.add(file.getPath()); + if (selection.wasAllPartitionsPruned()) { + // if all partitions were previously pruned, we only need to read 1 file (for the schema) + fileSet.add(this.parquetTableMetadata.getFiles().get(0).getPath()); + } else { + for (Metadata.ParquetFileMetadata file : this.parquetTableMetadata.getFiles()) { + fileSet.add(file.getPath()); + } } } else { // we need to expand the files from fileStatuses @@ -597,7 +602,6 @@ public long getRowCount() { final Path metaPath = new Path(status.getPath(), Metadata.METADATA_FILENAME); final Metadata.ParquetTableMetadataBase metadata = Metadata.readBlockMeta(fs, metaPath.toString()); for (Metadata.ParquetFileMetadata file : metadata.getFiles()) { - // fileNames.add(file.getPath()); fileSet.add(file.getPath()); } } else { @@ -614,8 +618,8 @@ public long getRowCount() { List fileNames = Lists.newArrayList(fileSet); - // when creating the file selection, set the selection root in the form /a/b instead of - // file:/a/b. The reason is that the file names above have been created in the form + // when creating the file selection, set the selection root without the URI prefix + // The reason is that the file names above have been created in the form // /a/b/c.parquet and the format of the selection root must match that of the file names // otherwise downstream operations such as partition pruning can break. final Path metaRootPath = Path.getPathWithoutSchemeAndAuthority(new Path(selection.getSelectionRoot())); @@ -625,14 +629,15 @@ public long getRowCount() { // because create() changes the root to include the scheme and authority; In future, if create() // is the preferred way to instantiate a file selection, we may need to do something different... // WARNING: file statuses and file names are inconsistent - FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString(), cacheFileRoot); + FileSelection newSelection = new FileSelection(selection.getStatuses(fs), fileNames, metaRootPath.toString(), + cacheFileRoot, selection.wasAllPartitionsPruned()); newSelection.setExpandedFully(); return newSelection; } private void init() throws IOException { - if (entries.size() == 1) { + if (entries.size() == 1 && parquetTableMetadata == null) { Path p = Path.getPathWithoutSchemeAndAuthority(new Path(entries.get(0).getPath())); Path metaPath = null; if (fs.isDirectory(p)) { @@ -642,9 +647,7 @@ private void init() throws IOException { } if (metaPath != null && fs.exists(metaPath)) { usedMetadataCache = true; - if (parquetTableMetadata == null) { - parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath.toString()); - } + parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath.toString()); } else { parquetTableMetadata = Metadata.getParquetTableMetadata(fs, p.toString()); } @@ -848,11 +851,11 @@ public String getDigest() { public String toString() { String cacheFileString = ""; if (usedMetadataCache) { - // for EXPLAIN it is useful to know what was the root of the cache file; - // if the cacheFileRoot is null (depending on whether single partition - // optimization was applied), then we would have read the cache file from - // selectionRoot - String str = (cacheFileRoot == null) ? selectionRoot : cacheFileRoot; + // For EXPLAIN, remove the URI prefix from cacheFileRoot. If cacheFileRoot is null, we + // would have read the cache file from selectionRoot + String str = (cacheFileRoot == null) ? + Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString() : + Path.getPathWithoutSchemeAndAuthority(new Path(cacheFileRoot)).toString(); cacheFileString = ", cacheFileRoot=" + str; } return "ParquetGroupScan [entries=" + entries @@ -913,7 +916,7 @@ public GroupScan applyLimit(long maxRecords) { } try { - FileSelection newSelection = new FileSelection(null, Lists.newArrayList(fileNames), getSelectionRoot(), cacheFileRoot); + FileSelection newSelection = new FileSelection(null, Lists.newArrayList(fileNames), getSelectionRoot(), cacheFileRoot, false); logger.debug("applyLimit() reduce parquet file # from {} to {}", fileSet.size(), fileNames.size()); return this.clone(newSelection); } catch (IOException e) { 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 0afd9259619..d6029392cf7 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 @@ -69,7 +69,7 @@ public void testPartitionPruningWithMetadataCache_1() throws Exception { String usedMetaPattern = "usedMetadataFile=true"; // since there are 2 or more sub-partitions the single partition cache file optimization does not apply // and cacheFileRoot should point to the top level selectionRoot - String cacheFileRootPattern = String.format("%s/%s", getDfsTestTmpSchemaLocation(), tableName1); + String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), tableName1); PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } @@ -88,7 +88,7 @@ public void testPartitionPruningWithMetadataCache_2() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - String cacheFileRootPattern = String.format("%s/%s/1994", getDfsTestTmpSchemaLocation(), tableName1); + String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s/1994", getDfsTestTmpSchemaLocation(), tableName1); PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } @@ -234,7 +234,7 @@ public void testDrill4530_1() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - String cacheFileRootPattern = String.format("%s/%s/1995/Q3", getDfsTestTmpSchemaLocation(), tableName2); + String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s/1995/Q3", getDfsTestTmpSchemaLocation(), tableName2); PlanTestBase.testPlanMatchingPatterns(query1, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } @@ -256,7 +256,7 @@ public void testDrill4530_2() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - String cacheFileRootPattern = String.format("%s/%s/1995", getDfsTestTmpSchemaLocation(), tableName2); + String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s/1995", getDfsTestTmpSchemaLocation(), tableName2); PlanTestBase.testPlanMatchingPatterns(query1, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } @@ -278,11 +278,33 @@ public void testDrill4530_3() throws Exception { assertEquals(expectedRowCount, actualRowCount); String numFilesPattern = "numFiles=" + expectedNumFiles; String usedMetaPattern = "usedMetadataFile=true"; - String cacheFileRootPattern = String.format("%s/%s", getDfsTestTmpSchemaLocation(), tableName2); + String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), tableName2); PlanTestBase.testPlanMatchingPatterns(query1, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern}, new String[] {"Filter"}); } + @Test // DRILL-4530 // non-existent partition (1 subdirectory's cache file will still be read for schema) + public void testDrill4530_4() throws Exception { + // create metadata cache + test(String.format("refresh table metadata dfs_test.`%s/%s`", getDfsTestTmpSchemaLocation(), tableName2)); + checkForMetadataFile(tableName2); + + // run query and check correctness + String query1 = String.format("select dir0, dir1, o_custkey, o_orderdate from dfs_test.`%s/%s` " + + " where dir0=1995 and dir1='Q6'", + getDfsTestTmpSchemaLocation(), tableName2); + int expectedRowCount = 0; + int expectedNumFiles = 1; + + int actualRowCount = testSql(query1); + assertEquals(expectedRowCount, actualRowCount); + String numFilesPattern = "numFiles=" + expectedNumFiles; + String usedMetaPattern = "usedMetadataFile=true"; + String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s/1995/Q2", getDfsTestTmpSchemaLocation(), tableName2); + PlanTestBase.testPlanMatchingPatterns(query1, 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);