From 4a2541206db911e9631a359aae09a38f6cef065e Mon Sep 17 00:00:00 2001 From: Arina Ielchiieva Date: Tue, 20 Mar 2018 18:29:45 +0000 Subject: [PATCH] DRILL-6331: Revisit Hive Drill native parquet implementation to be exposed to Drill optimizations (filter / limit push down, count to direct scan) 1. Factored out common logic for Drill parquet reader and Hive Drill native parquet readers: AbstractParquetGroupScan, AbstractParquetRowGroupScan, AbstractParquetScanBatchCreator. 2. Rules that worked previously only with ParquetGroupScan, now can be applied for any class that extends AbstractParquetGroupScan: DrillFilterItemStarReWriterRule, ParquetPruneScanRule, PruneScanRule. 3. Hive populated partition values based on information returned from Hive metastore. Drill populates partition values based on path difference between selection root and actual file path. Before ColumnExplorer populated partition values based on Drill approach. Since now ColumnExplorer populates values for parquet files from Hive tables, `populateImplicitColumns` method logic was changed to populated partition columns only based on given partition values. 4. Refactored ParquetPartitionDescriptor to be responsible for populating partition values rather than storing this logic in parquet group scan class. 5. Metadata class was moved to separate metadata package (org.apache.drill.exec.store.parquet.metadata). Factored out several inner classed to improve code readability. 6. Collected all Drill native parquet reader unit tests into one class TestHiveDrillNativeParquetReader, also added new tests to cover new functionality. 7. Reduced excessive logging when parquet files metadata is read. --- .../client/src/protobuf/UserBitShared.pb.cc | 14 +- .../client/src/protobuf/UserBitShared.pb.h | 5 +- ...vertHiveParquetScanToDrillParquetScan.java | 57 +- .../HiveDrillNativeParquetRowGroupScan.java | 131 ++ .../hive/HiveDrillNativeParquetScan.java | 204 +- ...iveDrillNativeParquetScanBatchCreator.java | 79 + .../hive/HiveDrillNativeParquetSubScan.java | 55 - .../hive/HiveDrillNativeScanBatchCreator.java | 208 -- .../exec/store/hive/HiveMetadataProvider.java | 5 +- .../exec/store/hive/HivePartitionHolder.java | 95 + .../TestHiveDrillNativeParquetReader.java | 248 ++ .../drill/exec/TestHivePartitionPruning.java | 19 - .../drill/exec/TestHiveProjectPushDown.java | 13 - .../apache/drill/exec/hive/HiveTestBase.java | 2 +- .../drill/exec/hive/TestHiveStorage.java | 215 +- .../hive/TestInfoSchemaOnHiveStorage.java | 6 +- .../sql/hive/TestViewSupportOnHiveTables.java | 2 +- .../store/hive/HiveTestDataGenerator.java | 82 +- .../kv_native_ext/part_key=1/kv_1.parquet | Bin 0 -> 220 bytes .../external/part_key=2/kv_2.parquet | Bin 0 -> 220 bytes .../drill/exec/ops/BaseOperatorContext.java | 42 +- .../exec/physical/base/AbstractGroupScan.java | 14 + .../drill/exec/physical/base/GroupScan.java | 10 + .../drill/exec/physical/base/ScanStats.java | 9 +- .../planner/ParquetPartitionDescriptor.java | 331 ++- .../drill/exec/planner/PlannerPhase.java | 4 + .../DrillFilterItemStarReWriterRule.java | 8 +- .../partition/ParquetPruneScanRule.java | 16 +- .../logical/partition/PruneScanRule.java | 77 +- .../sql/handlers/RefreshMetadataHandler.java | 2 +- .../drill/exec/store/ColumnExplorer.java | 90 +- .../drill/exec/store/TimedRunnable.java | 16 +- .../drill/exec/store/dfs/FileSelection.java | 37 +- .../exec/store/dfs/ReadEntryFromHDFS.java | 4 +- .../exec/store/dfs/easy/EasyFormatPlugin.java | 6 +- .../store/ischema/InfoSchemaGroupScan.java | 2 +- .../parquet/AbstractParquetGroupScan.java | 463 ++++ .../parquet/AbstractParquetRowGroupScan.java | 90 + .../AbstractParquetScanBatchCreator.java | 186 ++ .../drill/exec/store/parquet/Metadata.java | 2007 ----------------- .../store/parquet/ParquetFormatPlugin.java | 3 +- .../exec/store/parquet/ParquetGroupScan.java | 1236 ++-------- .../parquet/ParquetGroupScanStatistics.java | 215 ++ .../store/parquet/ParquetPushDownFilter.java | 23 +- .../store/parquet/ParquetReaderUtility.java | 116 +- .../store/parquet/ParquetRowGroupScan.java | 111 +- .../parquet/ParquetScanBatchCreator.java | 160 +- .../exec/store/parquet/RowGroupInfo.java | 98 + .../exec/store/parquet/metadata/Metadata.java | 694 ++++++ .../store/parquet/metadata/MetadataBase.java | 142 ++ .../parquet/metadata/MetadataPathUtils.java | 122 + .../{ => metadata}/MetadataVersion.java | 3 +- .../store/parquet/metadata/Metadata_V1.java | 329 +++ .../store/parquet/metadata/Metadata_V2.java | 418 ++++ .../store/parquet/metadata/Metadata_V3.java | 470 ++++ .../metadata/ParquetTableMetadataDirs.java | 49 + .../stat/ParquetMetaStatCollector.java | 35 +- .../TestCorruptParquetDateCorrection.java | 2 +- .../exec/store/FormatPluginSerDeTest.java | 36 +- .../parquet/TestParquetMetadataCache.java | 2 + .../parquet/TestParquetMetadataVersion.java | 1 + .../drill/exec/proto/UserBitShared.java | 22 +- .../exec/proto/beans/CoreOperatorType.java | 4 +- .../src/main/protobuf/UserBitShared.proto | 1 + 64 files changed, 5098 insertions(+), 4048 deletions(-) create mode 100644 contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java create mode 100644 contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java delete mode 100644 contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetSubScan.java delete mode 100644 contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java create mode 100644 contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java create mode 100644 contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java create mode 100755 contrib/storage-hive/core/src/test/resources/external/kv_native_ext/part_key=1/kv_1.parquet create mode 100755 contrib/storage-hive/core/src/test/resources/external/part_key=2/kv_2.parquet create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java rename exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/{ => metadata}/MetadataVersion.java (99%) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc index c5b25279b02..afab5afb957 100644 --- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc +++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc @@ -749,7 +749,7 @@ void protobuf_AddDesc_UserBitShared_2eproto() { "agmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALL" "OCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\t" "CANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_" - "REQUESTED\020\006*\302\006\n\020CoreOperatorType\022\021\n\rSING" + "REQUESTED\020\006*\360\006\n\020CoreOperatorType\022\021\n\rSING" "LE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FIL" "TER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004" "\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDE" @@ -770,11 +770,12 @@ void protobuf_AddDesc_UserBitShared_2eproto() { "P_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_" "SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_S" "CAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006" - "UNNEST\020**g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000" - "\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020" - "\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org" - ".apache.drill.exec.protoB\rUserBitSharedH" - "\001", 5121); + "UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_PARQUET_RO" + "W_GROUP_SCAN\020+*g\n\nSaslStatus\022\020\n\014SASL_UNK" + "NOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRE" + "SS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B" + ".\n\033org.apache.drill.exec.protoB\rUserBitS" + "haredH\001", 5167); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "UserBitShared.proto", &protobuf_RegisterTypes); UserCredentials::default_instance_ = new UserCredentials(); @@ -938,6 +939,7 @@ bool CoreOperatorType_IsValid(int value) { case 40: case 41: case 42: + case 43: return true; default: return false; diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h index 1a699065757..66c4cf79865 100644 --- a/contrib/native/client/src/protobuf/UserBitShared.pb.h +++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h @@ -246,11 +246,12 @@ enum CoreOperatorType { KUDU_SUB_SCAN = 39, FLATTEN = 40, LATERAL_JOIN = 41, - UNNEST = 42 + UNNEST = 42, + HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN = 43 }; bool CoreOperatorType_IsValid(int value); const CoreOperatorType CoreOperatorType_MIN = SINGLE_SENDER; -const CoreOperatorType CoreOperatorType_MAX = UNNEST; +const CoreOperatorType CoreOperatorType_MAX = HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN; const int CoreOperatorType_ARRAYSIZE = CoreOperatorType_MAX + 1; const ::google::protobuf::EnumDescriptor* CoreOperatorType_descriptor(); diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java index a7322454bc1..3484ab32a65 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java @@ -37,6 +37,7 @@ import org.apache.drill.exec.planner.sql.DrillSqlOperator; import org.apache.drill.exec.store.StoragePluginOptimizerRule; import org.apache.drill.exec.store.hive.HiveDrillNativeParquetScan; +import org.apache.drill.exec.store.hive.HiveMetadataProvider; import org.apache.drill.exec.store.hive.HiveReadEntry; import org.apache.drill.exec.store.hive.HiveScan; import org.apache.drill.exec.store.hive.HiveTableWithColumnCache; @@ -84,12 +85,12 @@ private ConvertHiveParquetScanToDrillParquetScan() { * {@link MapredParquetInputFormat} * 4) No error occurred while checking for the above conditions. An error is logged as warning. * - * @param call + * @param call rule call * @return True if the rule can be applied. False otherwise */ @Override public boolean matches(RelOptRuleCall call) { - final DrillScanRel scanRel = (DrillScanRel) call.rel(0); + final DrillScanRel scanRel = call.rel(0); if (!(scanRel.getGroupScan() instanceof HiveScan) || ((HiveScan) scanRel.getGroupScan()).isNativeReader()) { return false; @@ -99,6 +100,10 @@ public boolean matches(RelOptRuleCall call) { final HiveConf hiveConf = hiveScan.getHiveConf(); final HiveTableWithColumnCache hiveTable = hiveScan.getHiveReadEntry().getTable(); + if (containsUnsupportedDataTypes(hiveTable)) { + return false; + } + final Class> tableInputFormat = getInputFormatFromSD(HiveUtilities.getTableMetadata(hiveTable), hiveScan.getHiveReadEntry(), hiveTable.getSd(), hiveConf); @@ -139,9 +144,9 @@ public boolean matches(RelOptRuleCall call) { /** * Get the input format from given {@link StorageDescriptor} - * @param properties - * @param hiveReadEntry - * @param sd + * @param properties table properties + * @param hiveReadEntry hive read entry + * @param sd storage descriptor * @return {@link InputFormat} class or null if a failure has occurred. Failure is logged as warning. */ private Class> getInputFormatFromSD(final Properties properties, @@ -166,25 +171,41 @@ public boolean matches(RelOptRuleCall call) { @Override public void onMatch(RelOptRuleCall call) { try { - final DrillScanRel hiveScanRel = (DrillScanRel) call.rel(0); + final DrillScanRel hiveScanRel = call.rel(0); final HiveScan hiveScan = (HiveScan) hiveScanRel.getGroupScan(); final PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner()); final String partitionColumnLabel = settings.getFsPartitionColumnLabel(); final Table hiveTable = hiveScan.getHiveReadEntry().getTable(); - checkForUnsupportedDataTypes(hiveTable); + final HiveReadEntry hiveReadEntry = hiveScan.getHiveReadEntry(); - final Map partitionColMapping = - getPartitionColMapping(hiveTable, partitionColumnLabel); + final HiveMetadataProvider hiveMetadataProvider = new HiveMetadataProvider(hiveScan.getUserName(), hiveReadEntry, hiveScan.getStoragePlugin().getHiveConf()); + final List logicalInputSplits = hiveMetadataProvider.getInputSplits(hiveReadEntry); + + if (logicalInputSplits.isEmpty()) { + // table is empty, use original scan + return; + } - final DrillScanRel nativeScanRel = createNativeScanRel(partitionColMapping, hiveScanRel); + final Map partitionColMapping = getPartitionColMapping(hiveTable, partitionColumnLabel); + final DrillScanRel nativeScanRel = createNativeScanRel(partitionColMapping, hiveScanRel, logicalInputSplits); if (hiveScanRel.getRowType().getFieldCount() == 0) { call.transformTo(nativeScanRel); } else { final DrillProjectRel projectRel = createProjectRel(hiveScanRel, partitionColMapping, nativeScanRel); call.transformTo(projectRel); } + + + /* + Drill native scan should take precedence over Hive since it's more efficient and faster. + Hive does not always give correct costing (i.e. for external tables Hive does not have number of rows + and we calculate them approximately). On the contrary, Drill calculates number of rows exactly + and thus Hive Scan can be chosen instead of Drill native scan because costings allegedly lower for Hive. + To ensure Drill native scan we'll be chosen, reduce Hive scan importance to 0. + */ + call.getPlanner().setImportance(hiveScanRel, 0.0); } catch (final Exception e) { logger.warn("Failed to convert HiveScan to HiveDrillNativeParquetScan", e); } @@ -208,7 +229,8 @@ private Map getPartitionColMapping(final Table hiveTable, final * Helper method which creates a DrillScalRel with native HiveScan. */ private DrillScanRel createNativeScanRel(final Map partitionColMapping, - final DrillScanRel hiveScanRel) throws Exception{ + final DrillScanRel hiveScanRel, + final List logicalInputSplits) throws Exception { final RelDataTypeFactory typeFactory = hiveScanRel.getCluster().getTypeFactory(); final RelDataType varCharType = typeFactory.createSqlType(SqlTypeName.VARCHAR); @@ -245,10 +267,9 @@ private DrillScanRel createNativeScanRel(final Map partitionColM final HiveDrillNativeParquetScan nativeHiveScan = new HiveDrillNativeParquetScan( hiveScan.getUserName(), - hiveScan.getHiveReadEntry(), - hiveScan.getStoragePlugin(), nativeScanCols, - null); + hiveScan.getStoragePlugin(), + logicalInputSplits); return new DrillScanRel( hiveScanRel.getCluster(), @@ -321,15 +342,17 @@ private RexNode createPartitionColumnCast(final DrillScanRel hiveScanRel, final return rb.makeCast(outputType, inputRef); } - private void checkForUnsupportedDataTypes(final Table hiveTable) { - for(FieldSchema hiveField : hiveTable.getSd().getCols()) { + private boolean containsUnsupportedDataTypes(final Table hiveTable) { + for (FieldSchema hiveField : hiveTable.getSd().getCols()) { final Category category = TypeInfoUtils.getTypeInfoFromTypeString(hiveField.getType()).getCategory(); if (category == Category.MAP || category == Category.STRUCT || category == Category.UNION || category == Category.LIST) { - HiveUtilities.throwUnsupportedHiveDataTypeError(category.toString()); + logger.debug("Hive table contains unsupported data type: {}", category); + return true; } } + return false; } } diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java new file mode 100644 index 00000000000..e22701511a1 --- /dev/null +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.hive; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType; +import org.apache.drill.exec.store.StoragePluginRegistry; +import org.apache.drill.exec.store.parquet.AbstractParquetRowGroupScan; +import org.apache.drill.exec.store.parquet.RowGroupReadEntry; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher; +import org.apache.hadoop.mapred.JobConf; + +import java.io.IOException; +import java.util.List; + +@JsonTypeName("hive-drill-native-parquet-row-group-scan") +public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupScan { + + private final HiveStoragePlugin hiveStoragePlugin; + private final HiveStoragePluginConfig hiveStoragePluginConfig; + private final HivePartitionHolder hivePartitionHolder; + + @JsonCreator + public HiveDrillNativeParquetRowGroupScan(@JacksonInject StoragePluginRegistry registry, + @JsonProperty("userName") String userName, + @JsonProperty("hiveStoragePluginConfig") HiveStoragePluginConfig hiveStoragePluginConfig, + @JsonProperty("rowGroupReadEntries") List rowGroupReadEntries, + @JsonProperty("columns") List columns, + @JsonProperty("hivePartitionHolder") HivePartitionHolder hivePartitionHolder, + @JsonProperty("filter") LogicalExpression filter) throws ExecutionSetupException { + this(userName, + (HiveStoragePlugin) registry.getPlugin(hiveStoragePluginConfig), + rowGroupReadEntries, + columns, + hivePartitionHolder, + filter); + } + + public HiveDrillNativeParquetRowGroupScan(String userName, + HiveStoragePlugin hiveStoragePlugin, + List rowGroupReadEntries, + List columns, + HivePartitionHolder hivePartitionHolder, + LogicalExpression filter) { + super(userName, rowGroupReadEntries, columns, filter); + this.hiveStoragePlugin = Preconditions.checkNotNull(hiveStoragePlugin, "Could not find format config for the given configuration"); + this.hiveStoragePluginConfig = hiveStoragePlugin.getConfig(); + this.hivePartitionHolder = hivePartitionHolder; + } + + @JsonProperty + public HiveStoragePluginConfig getHiveStoragePluginConfig() { + return hiveStoragePluginConfig; + } + + @JsonProperty + public HivePartitionHolder getHivePartitionHolder() { + return hivePartitionHolder; + } + + @JsonIgnore + public HiveStoragePlugin getHiveStoragePlugin() { + return hiveStoragePlugin; + } + + @Override + public PhysicalOperator getNewWithChildren(List children) { + Preconditions.checkArgument(children.isEmpty()); + return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder, filter); + } + + @Override + public int getOperatorType() { + return CoreOperatorType.HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN_VALUE; + } + + @Override + public AbstractParquetRowGroupScan copy(List columns) { + return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder, filter); + } + + @Override + public boolean areCorruptDatesAutoCorrected() { + return true; + } + + @Override + public Configuration getFsConf(RowGroupReadEntry rowGroupReadEntry) throws IOException { + Path path = new Path(rowGroupReadEntry.getPath()).getParent(); + return new ProjectionPusher().pushProjectionsAndFilters( + new JobConf(hiveStoragePlugin.getHiveConf()), + path.getParent()); + } + + @Override + public boolean supportsFileImplicitColumns() { + return false; + } + + @Override + public List getPartitionValues(RowGroupReadEntry rowGroupReadEntry) { + return hivePartitionHolder.get(rowGroupReadEntry.getPath()); + } +} + diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java index 202bd435e1c..03a80d33861 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java @@ -21,94 +21,204 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.common.expression.LogicalExpression; import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.common.expression.ValueExpressions; import org.apache.drill.exec.physical.base.GroupScan; import org.apache.drill.exec.physical.base.PhysicalOperator; -import org.apache.drill.exec.physical.base.ScanStats; import org.apache.drill.exec.physical.base.SubScan; +import org.apache.drill.exec.proto.CoordinationProtos; import org.apache.drill.exec.store.StoragePluginRegistry; -import org.apache.drill.exec.store.hive.HiveTableWrapper.HivePartitionWrapper; +import org.apache.drill.exec.store.dfs.FileSelection; +import org.apache.drill.exec.store.dfs.ReadEntryWithPath; +import org.apache.drill.exec.store.hive.HiveMetadataProvider.LogicalInputSplit; +import org.apache.drill.exec.store.parquet.AbstractParquetGroupScan; +import org.apache.drill.exec.store.parquet.RowGroupReadEntry; +import org.apache.drill.exec.store.parquet.metadata.Metadata; +import org.apache.drill.exec.store.parquet.ParquetFormatConfig; +import org.apache.drill.exec.store.parquet.RowGroupInfo; +import org.apache.drill.exec.util.ImpersonationUtil; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; -/** - * Extension of {@link HiveScan} which support reading Hive tables using Drill's native parquet reader. - */ @JsonTypeName("hive-drill-native-parquet-scan") -public class HiveDrillNativeParquetScan extends HiveScan { +public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan { + + private final HiveStoragePlugin hiveStoragePlugin; + private HivePartitionHolder hivePartitionHolder; @JsonCreator - public HiveDrillNativeParquetScan(@JsonProperty("userName") String userName, - @JsonProperty("hiveReadEntry") HiveReadEntry hiveReadEntry, + public HiveDrillNativeParquetScan(@JacksonInject StoragePluginRegistry engineRegistry, + @JsonProperty("userName") String userName, @JsonProperty("hiveStoragePluginConfig") HiveStoragePluginConfig hiveStoragePluginConfig, @JsonProperty("columns") List columns, - @JacksonInject StoragePluginRegistry pluginRegistry) throws ExecutionSetupException { - super(userName, hiveReadEntry, hiveStoragePluginConfig, columns, pluginRegistry); + @JsonProperty("entries") List entries, + @JsonProperty("hivePartitionHolder") HivePartitionHolder hivePartitionHolder, + @JsonProperty("filter") LogicalExpression filter) throws IOException, ExecutionSetupException { + super(ImpersonationUtil.resolveUserName(userName), columns, entries, filter); + this.hiveStoragePlugin = (HiveStoragePlugin) engineRegistry.getPlugin(hiveStoragePluginConfig); + this.hivePartitionHolder = hivePartitionHolder; + + init(); } - public HiveDrillNativeParquetScan(String userName, HiveReadEntry hiveReadEntry, HiveStoragePlugin hiveStoragePlugin, - List columns, HiveMetadataProvider metadataProvider) throws ExecutionSetupException { - super(userName, hiveReadEntry, hiveStoragePlugin, columns, metadataProvider); + public HiveDrillNativeParquetScan(String userName, + List columns, + HiveStoragePlugin hiveStoragePlugin, + List logicalInputSplits) throws IOException { + this(userName, columns, hiveStoragePlugin, logicalInputSplits, ValueExpressions.BooleanExpression.TRUE); } - public HiveDrillNativeParquetScan(final HiveScan hiveScan) { - super(hiveScan); + public HiveDrillNativeParquetScan(String userName, + List columns, + HiveStoragePlugin hiveStoragePlugin, + List logicalInputSplits, + LogicalExpression filter) throws IOException { + super(userName, columns, new ArrayList<>(), filter); + + this.hiveStoragePlugin = hiveStoragePlugin; + this.hivePartitionHolder = new HivePartitionHolder(); + + for (LogicalInputSplit logicalInputSplit : logicalInputSplits) { + Iterator iterator = logicalInputSplit.getInputSplits().iterator(); + // logical input split contains list of splits by files + // we need to read path of only one to get file path + assert iterator.hasNext(); + InputSplit split = iterator.next(); + assert split instanceof FileSplit; + FileSplit fileSplit = (FileSplit) split; + Path finalPath = fileSplit.getPath(); + String pathString = Path.getPathWithoutSchemeAndAuthority(finalPath).toString(); + entries.add(new ReadEntryWithPath(pathString)); + + // store partition values per path + Partition partition = logicalInputSplit.getPartition(); + if (partition != null) { + hivePartitionHolder.add(pathString, partition.getValues()); + } + } + + init(); } - @Override - public ScanStats getScanStats() { - final ScanStats nativeHiveScanStats = super.getScanStats(); + private HiveDrillNativeParquetScan(HiveDrillNativeParquetScan that) { + super(that); + this.hiveStoragePlugin = that.hiveStoragePlugin; + this.hivePartitionHolder = that.hivePartitionHolder; + } - // As Drill's native parquet record reader is faster and memory efficient. Divide the CPU cost - // by a factor to let the planner choose HiveDrillNativeScan over HiveScan with SerDes. - return new ScanStats( - nativeHiveScanStats.getGroupScanProperty(), - nativeHiveScanStats.getRecordCount(), - nativeHiveScanStats.getCpuCost()/getSerDeOverheadFactor(), - nativeHiveScanStats.getDiskCost()); + @JsonProperty + public HiveStoragePluginConfig getHiveStoragePluginConfig() { + return hiveStoragePlugin.getConfig(); } - @Override - public SubScan getSpecificScan(int minorFragmentId) throws ExecutionSetupException { - try { - return new HiveDrillNativeParquetSubScan((HiveSubScan)super.getSpecificScan(minorFragmentId)); - } catch (IOException | ReflectiveOperationException e) { - throw new ExecutionSetupException(e); - } + @JsonProperty + public HivePartitionHolder getHivePartitionHolder() { + return hivePartitionHolder; } @Override - public boolean isNativeReader() { - return true; + public SubScan getSpecificScan(int minorFragmentId) { + List readEntries = getReadEntries(minorFragmentId); + HivePartitionHolder subPartitionHolder = new HivePartitionHolder(); + for (RowGroupReadEntry readEntry : readEntries) { + List values = hivePartitionHolder.get(readEntry.getPath()); + subPartitionHolder.add(readEntry.getPath(), values); + } + return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, readEntries, columns, subPartitionHolder, filter); } @Override - public PhysicalOperator getNewWithChildren(List children) throws ExecutionSetupException { + public PhysicalOperator getNewWithChildren(List children) { + Preconditions.checkArgument(children.isEmpty()); return new HiveDrillNativeParquetScan(this); } @Override - public HiveScan clone(HiveReadEntry hiveReadEntry) throws ExecutionSetupException { - return new HiveDrillNativeParquetScan(getUserName(), hiveReadEntry, getStoragePlugin(), getColumns(), getMetadataProvider()); + public HiveDrillNativeParquetScan clone(FileSelection selection) throws IOException { + HiveDrillNativeParquetScan newScan = new HiveDrillNativeParquetScan(this); + newScan.modifyFileSelection(selection); + newScan.init(); + return newScan; } @Override public GroupScan clone(List columns) { - final HiveDrillNativeParquetScan scan = new HiveDrillNativeParquetScan(this); - scan.columns = columns; - return scan; + HiveDrillNativeParquetScan newScan = new HiveDrillNativeParquetScan(this); + newScan.columns = columns; + return newScan; } @Override public String toString() { - final List partitions = getHiveReadEntry().getHivePartitionWrappers(); - int numPartitions = partitions == null ? 0 : partitions.size(); - return "HiveDrillNativeParquetScan [table=" + getHiveReadEntry().getHiveTableWrapper() - + ", columns=" + getColumns() - + ", numPartitions=" + numPartitions - + ", partitions= " + partitions - + ", inputDirectories=" + getMetadataProvider().getInputDirectories(getHiveReadEntry()) + "]"; + StringBuilder builder = new StringBuilder(); + builder.append("HiveDrillNativeParquetScan ["); + builder.append("entries=").append(entries); + builder.append(", numFiles=").append(getEntries().size()); + builder.append(", numRowGroups=").append(rowGroupInfos.size()); + + String filterString = getFilterString(); + if (!filterString.isEmpty()) { + builder.append(", filter=").append(filterString); + } + + builder.append(", columns=").append(columns); + builder.append("]"); + + return builder.toString(); } + + @Override + protected void initInternal() throws IOException { + ParquetFormatConfig formatConfig = new ParquetFormatConfig(); + Map fileStatusConfMap = new LinkedHashMap<>(); + for (ReadEntryWithPath entry : entries) { + Path path = new Path(entry.getPath()); + Configuration conf = new ProjectionPusher().pushProjectionsAndFilters( + new JobConf(hiveStoragePlugin.getHiveConf()), + path.getParent()); + FileSystem fs = path.getFileSystem(conf); + fileStatusConfMap.put(fs.getFileStatus(Path.getPathWithoutSchemeAndAuthority(path)), fs); + } + parquetTableMetadata = Metadata.getParquetTableMetadata(fileStatusConfMap, formatConfig); + } + + @Override + protected Collection getDrillbits() { + return hiveStoragePlugin.getContext().getBits(); + } + + @Override + protected AbstractParquetGroupScan cloneWithFileSelection(Collection filePaths) throws IOException { + FileSelection newSelection = new FileSelection(null, new ArrayList<>(filePaths), null, null, false); + return clone(newSelection); + } + + @Override + protected boolean supportsFileImplicitColumns() { + return false; + } + + @Override + protected List getPartitionValues(RowGroupInfo rowGroupInfo) { + return hivePartitionHolder.get(rowGroupInfo.getPath()); + } + } diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java new file mode 100644 index 00000000000..669369b3ed9 --- /dev/null +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScanBatchCreator.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.hive; + +import com.google.common.base.Preconditions; +import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.exec.ops.ExecutorFragmentContext; +import org.apache.drill.exec.ops.OperatorContext; +import org.apache.drill.exec.physical.impl.BatchCreator; +import org.apache.drill.exec.record.CloseableRecordBatch; +import org.apache.drill.exec.record.RecordBatch; +import org.apache.drill.exec.server.options.OptionManager; +import org.apache.drill.exec.store.dfs.DrillFileSystem; +import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class HiveDrillNativeParquetScanBatchCreator extends AbstractParquetScanBatchCreator implements BatchCreator { + + @Override + public CloseableRecordBatch getBatch(ExecutorFragmentContext context, HiveDrillNativeParquetRowGroupScan rowGroupScan, List children) throws ExecutionSetupException { + Preconditions.checkArgument(children.isEmpty()); + OperatorContext oContext = context.newOperatorContext(rowGroupScan); + return getBatch(context, rowGroupScan, oContext); + } + + @Override + protected AbstractDrillFileSystemManager getDrillFileSystemCreator(OperatorContext operatorContext, OptionManager optionManager) { + return new HiveDrillNativeParquetDrillFileSystemManager(operatorContext); + } + + /** + * Caches file system per path and returns file system from cache if it exists there. + * Creates only non-tracking file systems. + */ + private class HiveDrillNativeParquetDrillFileSystemManager extends AbstractDrillFileSystemManager { + + private final Map fileSystems; + + HiveDrillNativeParquetDrillFileSystemManager(OperatorContext operatorContext) { + super(operatorContext); + this.fileSystems = new HashMap<>(); + } + + @Override + protected DrillFileSystem get(Configuration config, String path) throws ExecutionSetupException { + DrillFileSystem fs = fileSystems.get(path); + if (fs == null) { + try { + fs = operatorContext.newNonTrackingFileSystem(config); + } catch (IOException e) { + throw new ExecutionSetupException(String.format("Failed to create non-tracking DrillFileSystem: %s", e.getMessage()), e); + } + fileSystems.put(path, fs); + } + return fs; + } + } + +} diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetSubScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetSubScan.java deleted file mode 100644 index 2129ed45423..00000000000 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetSubScan.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.drill.exec.store.hive; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.drill.common.exceptions.ExecutionSetupException; -import org.apache.drill.common.expression.SchemaPath; -import org.apache.drill.exec.store.StoragePluginRegistry; - -import java.io.IOException; -import java.util.List; - -/** - * Extension of {@link HiveSubScan} which support reading Hive tables using Drill's native parquet reader. - */ -@JsonTypeName("hive-drill-native-parquet-sub-scan") -public class HiveDrillNativeParquetSubScan extends HiveSubScan { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveDrillNativeParquetSubScan.class); - - @JsonCreator - public HiveDrillNativeParquetSubScan(@JacksonInject StoragePluginRegistry registry, - @JsonProperty("userName") String userName, - @JsonProperty("splits") List> splits, - @JsonProperty("hiveReadEntry") HiveReadEntry hiveReadEntry, - @JsonProperty("splitClasses") List splitClasses, - @JsonProperty("columns") List columns, - @JsonProperty("hiveStoragePluginConfig") HiveStoragePluginConfig hiveStoragePluginConfig) - throws IOException, ExecutionSetupException, ReflectiveOperationException { - super(registry, userName, splits, hiveReadEntry, splitClasses, columns, hiveStoragePluginConfig); - } - - public HiveDrillNativeParquetSubScan(final HiveSubScan subScan) - throws IOException, ExecutionSetupException, ReflectiveOperationException { - super(subScan.getUserName(), subScan.getSplits(), subScan.getHiveReadEntry(), subScan.getSplitClasses(), - subScan.getColumns(), subScan.getStoragePlugin()); - } -} diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java deleted file mode 100644 index 43318d17cc5..00000000000 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.drill.exec.store.hive; - -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import com.google.common.base.Functions; -import org.apache.drill.common.AutoCloseables; -import org.apache.drill.common.exceptions.ExecutionSetupException; -import org.apache.drill.common.expression.SchemaPath; -import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.ops.ExecutorFragmentContext; -import org.apache.drill.exec.ops.OperatorContext; -import org.apache.drill.exec.physical.impl.BatchCreator; -import org.apache.drill.exec.physical.impl.ScanBatch; -import org.apache.drill.exec.record.RecordBatch; -import org.apache.drill.exec.store.RecordReader; -import org.apache.drill.exec.store.hive.readers.HiveDefaultReader; -import org.apache.drill.exec.store.parquet.ParquetDirectByteBufferAllocator; -import org.apache.drill.exec.store.parquet.ParquetReaderUtility; -import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader; -import org.apache.drill.exec.util.ImpersonationUtil; -import org.apache.drill.exec.util.Utilities; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.parquet.hadoop.CodecFactory; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -@SuppressWarnings("unused") -public class HiveDrillNativeScanBatchCreator implements BatchCreator { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveDrillNativeScanBatchCreator.class); - - @Override - public ScanBatch getBatch(ExecutorFragmentContext context, HiveDrillNativeParquetSubScan config, List children) - throws ExecutionSetupException { - final HiveTableWithColumnCache table = config.getTable(); - final List> splits = config.getInputSplits(); - final List partitions = config.getPartitions(); - final List columns = config.getColumns(); - final String partitionDesignator = context.getOptions() - .getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val; - List> implicitColumns = Lists.newLinkedList(); - boolean selectAllQuery = Utilities.isStarQuery(columns); - - final boolean hasPartitions = (partitions != null && partitions.size() > 0); - - final List partitionColumns = Lists.newArrayList(); - final List selectedPartitionColumns = Lists.newArrayList(); - List tableColumns = columns; - if (!selectAllQuery) { - // Separate out the partition and non-partition columns. Non-partition columns are passed directly to the - // ParquetRecordReader. Partition columns are passed to ScanBatch. - tableColumns = Lists.newArrayList(); - Pattern pattern = Pattern.compile(String.format("%s[0-9]+", partitionDesignator)); - for (SchemaPath column : columns) { - Matcher m = pattern.matcher(column.getRootSegmentPath()); - if (m.matches()) { - selectedPartitionColumns.add( - Integer.parseInt(column.getRootSegmentPath().substring(partitionDesignator.length()))); - } else { - tableColumns.add(column); - } - } - } - - final OperatorContext oContext = context.newOperatorContext(config); - - int currentPartitionIndex = 0; - final List readers = new LinkedList<>(); - - final HiveConf conf = config.getHiveConf(); - - // TODO: In future we can get this cache from Metadata cached on filesystem. - final Map footerCache = Maps.newHashMap(); - - Map mapWithMaxColumns = Maps.newLinkedHashMap(); - try { - for (List splitGroups : splits) { - for (InputSplit split : splitGroups) { - final FileSplit fileSplit = (FileSplit) split; - final Path finalPath = fileSplit.getPath(); - final JobConf cloneJob = - new ProjectionPusher().pushProjectionsAndFilters(new JobConf(conf), finalPath.getParent()); - final FileSystem fs = finalPath.getFileSystem(cloneJob); - - ParquetMetadata parquetMetadata = footerCache.get(finalPath.toString()); - if (parquetMetadata == null) { - parquetMetadata = ParquetFileReader.readFooter(cloneJob, finalPath); - footerCache.put(finalPath.toString(), parquetMetadata); - } - final List rowGroupNums = getRowGroupNumbersFromFileSplit(fileSplit, parquetMetadata); - - for (int rowGroupNum : rowGroupNums) { - //DRILL-5009 : Skip the row group if the row count is zero - if (parquetMetadata.getBlocks().get(rowGroupNum).getRowCount() == 0) { - continue; - } - // Drill has only ever written a single row group per file, only detect corruption - // in the first row group - ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = - ParquetReaderUtility.detectCorruptDates(parquetMetadata, config.getColumns(), true); - if (logger.isDebugEnabled()) { - logger.debug(containsCorruptDates.toString()); - } - readers.add(new ParquetRecordReader( - context, - Path.getPathWithoutSchemeAndAuthority(finalPath).toString(), - rowGroupNum, fs, - CodecFactory.createDirectCodecFactory(fs.getConf(), - new ParquetDirectByteBufferAllocator(oContext.getAllocator()), 0), - parquetMetadata, - tableColumns, - containsCorruptDates) - ); - Map implicitValues = Maps.newLinkedHashMap(); - - if (hasPartitions) { - List values = partitions.get(currentPartitionIndex).getValues(); - for (int i = 0; i < values.size(); i++) { - if (selectAllQuery || selectedPartitionColumns.contains(i)) { - implicitValues.put(partitionDesignator + i, values.get(i)); - } - } - } - implicitColumns.add(implicitValues); - if (implicitValues.size() > mapWithMaxColumns.size()) { - mapWithMaxColumns = implicitValues; - } - } - currentPartitionIndex++; - } - } - } catch (final IOException|RuntimeException e) { - AutoCloseables.close(e, readers); - throw new ExecutionSetupException("Failed to create RecordReaders. " + e.getMessage(), e); - } - - // all readers should have the same number of implicit columns, add missing ones with value null - mapWithMaxColumns = Maps.transformValues(mapWithMaxColumns, Functions.constant((String) null)); - for (Map map : implicitColumns) { - map.putAll(Maps.difference(map, mapWithMaxColumns).entriesOnlyOnRight()); - } - - // If there are no readers created (which is possible when the table is empty or no row groups are matched), - // create an empty RecordReader to output the schema - if (readers.size() == 0) { - readers.add(new HiveDefaultReader(table, null, null, tableColumns, context, conf, - ImpersonationUtil.createProxyUgi(config.getUserName(), context.getQueryUserName()))); - } - - return new ScanBatch(context, oContext, readers, implicitColumns); - } - - /** - * Get the list of row group numbers for given file input split. Logic used here is same as how Hive's parquet input - * format finds the row group numbers for input split. - */ - private List getRowGroupNumbersFromFileSplit(final FileSplit split, - final ParquetMetadata footer) throws IOException { - final List blocks = footer.getBlocks(); - - final long splitStart = split.getStart(); - final long splitLength = split.getLength(); - - final List rowGroupNums = Lists.newArrayList(); - - int i = 0; - for (final BlockMetaData block : blocks) { - final long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); - if (firstDataPage >= splitStart && firstDataPage < splitStart + splitLength) { - rowGroupNums.add(i); - } - i++; - } - - return rowGroupNums; - } -} diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java index b11ef3b7a26..c8775643a13 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveMetadataProvider.java @@ -243,7 +243,10 @@ private HiveStats getStatsEstimateFromInputSplits(final List data += split.getLength(); } - return new HiveStats(data/RECORD_SIZE, data); + long numRows = data / RECORD_SIZE; + // if the result of division is zero and data size > 0, estimate to one row + numRows = numRows == 0 && data > 0 ? 1 : numRows; + return new HiveStats(numRows, data); } /** diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java new file mode 100644 index 00000000000..803144e0b2d --- /dev/null +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HivePartitionHolder.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.hive; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Helper class that stores partition values per key. + * Key to index mapper contains key and index corresponding to partition values position in partition values list. + * Since several keys may have that same partition values, such structure is optimized to save memory usage. + * Partition values are stored in list of consecutive values. + */ +public class HivePartitionHolder { + + private final Map keyToIndexMapper; + private final List> partitionValues; + + @JsonCreator + public HivePartitionHolder(@JsonProperty("keyToIndexMapper") Map keyToIndexMapper, + @JsonProperty("partitionValues") List> partitionValues) { + this.keyToIndexMapper = keyToIndexMapper; + this.partitionValues = partitionValues; + } + + public HivePartitionHolder() { + this.keyToIndexMapper = new HashMap<>(); + this.partitionValues = new ArrayList<>(); + } + + @JsonProperty + public Map getKeyToIndexMapper() { + return keyToIndexMapper; + } + + @JsonProperty + public List> getPartitionValues() { + return partitionValues; + } + + /** + * Checks if partition values already exist in holder. + * If not, adds them to holder and adds key and index corresponding to partition values to mapper. + * If partition values exist, adds key and existing partition values index to mapper. + * + * @param key mapper key + * @param values partition values + */ + public void add(String key, List values) { + int index = partitionValues.indexOf(values); + if (index == -1) { + index = partitionValues.size(); + partitionValues.add(values); + + } + keyToIndexMapper.put(key, index); + } + + /** + * Returns list of partition values stored in holder for the given key. + * If there are no corresponding partition values, return empty list. + * + * @param key mapper key + * @return list of partition values + */ + public List get(String key) { + Integer index = keyToIndexMapper.get(key); + if (index == null) { + return Collections.emptyList(); + } + return partitionValues.get(index); + } + +} diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java new file mode 100644 index 00000000000..23c67b5747c --- /dev/null +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveDrillNativeParquetReader.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec; + +import org.apache.drill.PlanTestBase; +import org.apache.drill.categories.HiveStorageTest; +import org.apache.drill.categories.SlowTest; +import org.apache.drill.common.exceptions.UserRemoteException; +import org.apache.drill.exec.hive.HiveTestBase; +import org.apache.drill.exec.planner.physical.PlannerSettings; +import org.hamcrest.CoreMatchers; +import org.joda.time.DateTime; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertEquals; + +@Category({SlowTest.class, HiveStorageTest.class}) +public class TestHiveDrillNativeParquetReader extends HiveTestBase { + + @BeforeClass + public static void init() { + setSessionOption(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS, true); + setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true); + } + + @AfterClass + public static void cleanup() { + resetSessionOption(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS); + resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY); + } + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testFilterPushDownForManagedTable() throws Exception { + String query = "select * from hive.kv_native where key > 1"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 2, actualRowCount); + + testPlanMatchingPatterns(query, + new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, null); + } + + @Test + public void testFilterPushDownForExternalTable() throws Exception { + String query = "select * from hive.kv_native_ext where key = 1"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 1, actualRowCount); + + testPlanMatchingPatterns(query, + new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, null); + } + + @Test + public void testManagedPartitionPruning() throws Exception { + String query = "select * from hive.readtest_parquet where tinyint_part = 64"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 2, actualRowCount); + + // Hive partition pruning is applied during logical stage + // while convert to Drill native parquet reader during physical + // thus plan should not contain filter + testPlanMatchingPatterns(query, + new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, new String[]{"Filter"}); + } + + @Test + public void testExternalPartitionPruning() throws Exception { + String query = "select `key` from hive.kv_native_ext where part_key = 2"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 2, actualRowCount); + + // Hive partition pruning is applied during logical stage + // while convert to Drill native parquet reader during physical + // thus plan should not contain filter + testPlanMatchingPatterns(query, + new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, new String[]{"Filter"}); + } + + @Test + public void testSimpleStarSubQueryFilterPushDown() throws Exception { + String query = "select * from (select * from (select * from hive.kv_native)) where key > 1"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 2, actualRowCount); + + testPlanMatchingPatterns(query, + new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, null); + } + + @Test + public void testPartitionedExternalTable() throws Exception { + String query = "select * from hive.kv_native_ext"; + + testPlanMatchingPatterns(query, new String[]{"HiveDrillNativeParquetScan", "numFiles=2"}, null); + + testBuilder() + .sqlQuery(query) + .unOrdered() + .baselineColumns("key", "part_key") + .baselineValues(1, 1) + .baselineValues(2, 1) + .baselineValues(3, 2) + .baselineValues(4, 2) + .go(); + } + + @Test + public void testEmptyTable() throws Exception { + String query = "select * from hive.empty_table"; + // Hive reader should be chosen to output the schema + testPlanMatchingPatterns(query, new String[]{"HiveScan"}, new String[]{"HiveDrillNativeParquetScan"}); + } + + @Test + public void testEmptyPartition() throws Exception { + String query = "select * from hive.kv_native_ext where part_key = 3"; + // Hive reader should be chosen to output the schema + testPlanMatchingPatterns(query, new String[]{"HiveScan"}, new String[]{"HiveDrillNativeParquetScan"}); + } + + @Test + public void testPhysicalPlanSubmission() throws Exception { + // checks only group scan + PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.kv_native"); + PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.kv_native_ext"); + } + + @Test + public void testProjectPushDownOptimization() throws Exception { + String query = "select boolean_field, int_part from hive.readtest_parquet"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 2, actualRowCount); + + testPlanMatchingPatterns(query, + // partition column is named during scan as Drill partition columns + // it will be renamed to actual value in subsequent project + new String[]{"Project\\(boolean_field=\\[\\$0\\], int_part=\\[CAST\\(\\$1\\):INTEGER\\]\\)", + "HiveDrillNativeParquetScan", + "columns=\\[`boolean_field`, `dir9`\\]"}, + new String[]{}); + } + + @Test + public void testLimitPushDownOptimization() throws Exception { + String query = "select * from hive.kv_native limit 2"; + + int actualRowCount = testSql(query); + assertEquals("Expected and actual row count should match", 2, actualRowCount); + + testPlanMatchingPatterns(query, new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, null); + } + + @Test + public void testConvertCountToDirectScanOptimization() throws Exception { + String query = "select count(1) as cnt from hive.kv_native"; + + testPlanMatchingPatterns(query, new String[]{"DynamicPojoRecordReader"}, null); + + testBuilder() + .sqlQuery(query) + .unOrdered() + .baselineColumns("cnt") + .baselineValues(8L) + .go(); + } + + @Test + public void testImplicitColumns() throws Exception { + thrown.expect(UserRemoteException.class); + thrown.expectMessage(CoreMatchers.allOf(containsString("VALIDATION ERROR"), containsString("not found in any table"))); + + test("select *, filename, fqn, filepath, suffix from hive.kv_native"); + } + + @Test // DRILL-3739 + public void testReadingFromStorageHandleBasedTable() throws Exception { + testBuilder() + .sqlQuery("select * from hive.kv_sh order by key limit 2") + .ordered() + .baselineColumns("key", "value") + .expectsEmptyResultSet() + .go(); + } + + @Test + public void testReadAllSupportedHiveDataTypesNativeParquet() throws Exception { + String query = "select * from hive.readtest_parquet"; + + testPlanMatchingPatterns(query, new String[] {"HiveDrillNativeParquetScan"}, null); + + testBuilder() + .sqlQuery(query) + .unOrdered() + .baselineColumns("binary_field", "boolean_field", "tinyint_field", "decimal0_field", "decimal9_field", "decimal18_field", "decimal28_field", "decimal38_field", "double_field", "float_field", "int_field", "bigint_field", "smallint_field", "string_field", "varchar_field", "timestamp_field", "char_field", + // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now. + //"binary_part", + "boolean_part", "tinyint_part", "decimal0_part", "decimal9_part", "decimal18_part", "decimal28_part", "decimal38_part", "double_part", "float_part", "int_part", "bigint_part", "smallint_part", "string_part", "varchar_part", "timestamp_part", "date_part", "char_part") + .baselineValues("binaryfield".getBytes(), false, 34, new BigDecimal("66"), new BigDecimal("2347.92"), new BigDecimal("2758725827.99990"), new BigDecimal("29375892739852.8"), new BigDecimal("89853749534593985.783"), 8.345d, 4.67f, 123456, 234235L, 3455, "stringfield", "varcharfield", new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), "charfield", + // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now. + //"binary", + true, 64, new BigDecimal("37"), new BigDecimal("36.90"), new BigDecimal("3289379872.94565"), new BigDecimal("39579334534534.4"), new BigDecimal("363945093845093890.900"), 8.345d, 4.67f, 123456, 234235L, 3455, "string", "varchar", new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), new DateTime(Date.valueOf("2013-07-05").getTime()), "char").baselineValues( // All fields are null, but partition fields have non-null values + null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, + // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now. + //"binary", + true, 64, new BigDecimal("37"), new BigDecimal("36.90"), new BigDecimal("3289379872.94565"), new BigDecimal("39579334534534.4"), new BigDecimal("363945093845093890.900"), 8.345d, 4.67f, 123456, 234235L, 3455, "string", "varchar", new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), new DateTime(Date.valueOf("2013-07-05").getTime()), "char").go(); + } + + @Test // DRILL-3938 + public void testNativeReaderIsDisabledForAlteredPartitionedTable() throws Exception { + String query = "select key, `value`, newcol from hive.kv_parquet order by key limit 1"; + + // Make sure the HiveScan in plan has no native parquet reader + testPlanMatchingPatterns(query, new String[] {"HiveScan"}, new String[]{"HiveDrillNativeParquetScan"}); + } + +} diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHivePartitionPruning.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHivePartitionPruning.java index ef5a169162c..7583f42ffcc 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHivePartitionPruning.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHivePartitionPruning.java @@ -120,25 +120,6 @@ public void pruneDataTypeSupport() throws Exception { assertFalse(plan.contains("Filter")); } - @Test - public void pruneDataTypeSupportNativeReaders() throws Exception { - try { - test(String.format("alter session set `%s` = true", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - final String query = "EXPLAIN PLAN FOR " + - "SELECT * FROM hive.readtest_parquet WHERE tinyint_part = 64"; - - final String plan = getPlanInString(query, OPTIQ_FORMAT); - - // Check and make sure that Filter is not present in the plan - assertFalse(plan.contains("Filter")); - - // Make sure the plan contains the Hive scan utilizing native parquet reader - assertTrue(plan.contains("groupscan=[HiveDrillNativeParquetScan")); - } finally { - test(String.format("alter session set `%s` = false", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - } - } - @Test // DRILL-3579 public void selectFromPartitionedTableWithNullPartitions() throws Exception { final String query = "SELECT count(*) nullCount FROM hive.partition_pruning_test " + diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java index 6d7ad136b42..26a16b92f57 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java @@ -18,7 +18,6 @@ package org.apache.drill.exec; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import org.apache.drill.categories.HiveStorageTest; import org.apache.drill.exec.hive.HiveTestBase; @@ -109,16 +108,4 @@ public void testHiveCountStar() throws Exception { testHelper(query, 1, expectedColNames); } - @Test - public void projectPushDownOnHiveParquetTable() throws Exception { - try { - test(String.format("alter session set `%s` = true", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - String query = "SELECT boolean_field, boolean_part, int_field, int_part FROM hive.readtest_parquet"; - String expectedColNames = "\"columns\" : [ \"`boolean_field`\", \"`dir0`\", \"`int_field`\", \"`dir9`\" ]"; - - testHelper(query, 2, expectedColNames, "hive-drill-native-parquet-scan"); - } finally { - test(String.format("alter session set `%s` = false", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - } - } } diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java index e3d884b5797..5758eec4a98 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java @@ -31,7 +31,7 @@ public class HiveTestBase extends PlanTestBase { @BeforeClass public static void generateHive() throws Exception { - hiveTest = HiveTestDataGenerator.getInstance(dirTestWatcher.getRootDir()); + hiveTest = HiveTestDataGenerator.getInstance(dirTestWatcher); hiveTest.addHiveTestPlugin(getDrillbitContext().getStorage()); } diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java index 4da22b6a3ae..1b7774878a1 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java @@ -31,8 +31,10 @@ import org.joda.time.DateTime; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; import java.math.BigDecimal; import java.sql.Date; @@ -42,38 +44,25 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @Category({SlowTest.class, HiveStorageTest.class}) public class TestHiveStorage extends HiveTestBase { + @BeforeClass - public static void setupOptions() throws Exception { - test(String.format("alter session set `%s` = true", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY)); + public static void init() { + setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true); } - @Test // DRILL-4083 - public void testNativeScanWhenNoColumnIsRead() throws Exception { - try { - test(String.format("alter session set `%s` = true", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - - String query = "SELECT count(*) as col FROM hive.countStar_Parquet"; - testPhysicalPlan(query, "hive-drill-native-parquet-scan"); - - testBuilder() - .sqlQuery(query) - .unOrdered() - .baselineColumns("col") - .baselineValues(200L) - .go(); - } finally { - test("alter session reset `%s`", - ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS); - } + @AfterClass + public static void cleanup() { + resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY); } + @Rule + public ExpectedException thrown = ExpectedException.none(); + @Test public void hiveReadWithDb() throws Exception { test("select * from hive.kv"); @@ -206,123 +195,6 @@ public void readAllSupportedHiveDataTypes() throws Exception { .go(); } - /** - * Test to ensure Drill reads the all supported types through native Parquet readers. - * NOTE: As part of Hive 1.2 upgrade, make sure this test and {@link #readAllSupportedHiveDataTypes()} are merged - * into one test. - */ - @Test - public void readAllSupportedHiveDataTypesNativeParquet() throws Exception { - try { - test(String.format("alter session set `%s` = true", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - final String query = "SELECT * FROM hive.readtest_parquet"; - - // Make sure the plan has Hive scan with native parquet reader - testPhysicalPlan(query, "hive-drill-native-parquet-scan"); - - testBuilder().sqlQuery(query) - .unOrdered() - .baselineColumns( - "binary_field", - "boolean_field", - "tinyint_field", - "decimal0_field", - "decimal9_field", - "decimal18_field", - "decimal28_field", - "decimal38_field", - "double_field", - "float_field", - "int_field", - "bigint_field", - "smallint_field", - "string_field", - "varchar_field", - "timestamp_field", - "char_field", - // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now. - //"binary_part", - "boolean_part", - "tinyint_part", - "decimal0_part", - "decimal9_part", - "decimal18_part", - "decimal28_part", - "decimal38_part", - "double_part", - "float_part", - "int_part", - "bigint_part", - "smallint_part", - "string_part", - "varchar_part", - "timestamp_part", - "date_part", - "char_part") - .baselineValues( - "binaryfield".getBytes(), - false, - 34, - new BigDecimal("66"), - new BigDecimal("2347.92"), - new BigDecimal("2758725827.99990"), - new BigDecimal("29375892739852.8"), - new BigDecimal("89853749534593985.783"), - 8.345d, - 4.67f, - 123456, - 234235L, - 3455, - "stringfield", - "varcharfield", - new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), - "charfield", - // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now. - //"binary", - true, - 64, - new BigDecimal("37"), - new BigDecimal("36.90"), - new BigDecimal("3289379872.94565"), - new BigDecimal("39579334534534.4"), - new BigDecimal("363945093845093890.900"), - 8.345d, - 4.67f, - 123456, - 234235L, - 3455, - "string", - "varchar", - new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), - new DateTime(Date.valueOf("2013-07-05").getTime()), - "char") - .baselineValues( // All fields are null, but partition fields have non-null values - null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, - // There is a regression in Hive 1.2.1 in binary and boolean partition columns. Disable for now. - //"binary", - true, - 64, - new BigDecimal("37"), - new BigDecimal("36.90"), - new BigDecimal("3289379872.94565"), - new BigDecimal("39579334534534.4"), - new BigDecimal("363945093845093890.900"), - 8.345d, - 4.67f, - 123456, - 234235L, - 3455, - "string", - "varchar", - new DateTime(Timestamp.valueOf("2013-07-05 17:01:00").getTime()), - new DateTime(Date.valueOf("2013-07-05").getTime()), - "char") - .go(); - } finally { - test(String.format("alter session set `%s` = false", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - } - } - @Test public void orderByOnHiveTable() throws Exception { testBuilder() @@ -402,19 +274,7 @@ public void readFromAlteredPartitionedTableWithEmptyGroupType() throws Exception .go(); } - @Test // DRILL-3938 - public void nativeReaderIsDisabledForAlteredPartitionedTable() throws Exception { - try { - test(String.format("alter session set `%s` = true", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - final String query = "EXPLAIN PLAN FOR SELECT key, `value`, newcol FROM hive.kv_parquet ORDER BY key LIMIT 1"; - // Make sure the HiveScan in plan has no native parquet reader - final String planStr = getPlanInString(query, JSON_FORMAT); - assertFalse("Hive native is not expected in the plan", planStr.contains("hive-drill-native-parquet-scan")); - } finally { - test(String.format("alter session set `%s` = false", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - } - } @Test // DRILL-3739 public void readingFromStorageHandleBasedTable() throws Exception { @@ -426,22 +286,6 @@ public void readingFromStorageHandleBasedTable() throws Exception { .go(); } - @Test // DRILL-3739 - public void readingFromStorageHandleBasedTable2() throws Exception { - try { - test(String.format("alter session set `%s` = true", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - - testBuilder() - .sqlQuery("SELECT * FROM hive.kv_sh ORDER BY key LIMIT 2") - .ordered() - .baselineColumns("key", "value") - .expectsEmptyResultSet() - .go(); - } finally { - test(String.format("alter session set `%s` = false", ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS)); - } - } - @Test // DRILL-3688 public void readingFromSmallTableWithSkipHeaderAndFooter() throws Exception { testBuilder() @@ -480,23 +324,20 @@ public void readingFromLargeTableWithSkipHeaderAndFooter() throws Exception { .go(); } - @Test // DRILL-3688 - public void testIncorrectHeaderFooterProperty() throws Exception { - Map testData = ImmutableMap.builder() - .put("hive.skipper.kv_incorrect_skip_header","skip.header.line.count") - .put("hive.skipper.kv_incorrect_skip_footer", "skip.footer.line.count") - .build(); - - String query = "select * from %s"; - String exceptionMessage = "Hive table property %s value 'A' is non-numeric"; - - for (Map.Entry entry : testData.entrySet()) { - try { - test(String.format(query, entry.getKey())); - } catch (UserRemoteException e) { - assertThat(e.getMessage(), containsString(String.format(exceptionMessage, entry.getValue()))); - } - } + @Test + public void testIncorrectHeaderProperty() throws Exception { + String query = "select * from hive.skipper.kv_incorrect_skip_header"; + thrown.expect(UserRemoteException.class); + thrown.expectMessage(containsString("Hive table property skip.header.line.count value 'A' is non-numeric")); + test(query); + } + + @Test + public void testIncorrectFooterProperty() throws Exception { + String query = "select * from hive.skipper.kv_incorrect_skip_footer"; + thrown.expect(UserRemoteException.class); + thrown.expectMessage(containsString("Hive table property skip.footer.line.count value 'A' is non-numeric")); + test(query); } @Test @@ -571,6 +412,7 @@ public void testNonAsciiStringLiterals() throws Exception { @Test public void testPhysicalPlanSubmission() throws Exception { PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.kv"); + PlanTestBase.testPhysicalPlanExecutionBasedOnQuery("select * from hive.readtest"); } private void verifyColumnsMetadata(List columnsList, Map expectedResult) { @@ -583,9 +425,4 @@ private void verifyColumnsMetadata(List columns assertTrue("Column should be nullable", columnMetadata.getIsNullable()); } } - - @AfterClass - public static void shutdownOptions() throws Exception { - test(String.format("alter session set `%s` = false", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY)); - } } diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java index 7f7a4011d76..80da976d443 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java @@ -46,9 +46,10 @@ public void showTablesFromDb() throws Exception{ .baselineValues("hive.default", "kv") .baselineValues("hive.default", "kv_parquet") .baselineValues("hive.default", "kv_sh") - .baselineValues("hive.default", "countstar_parquet") .baselineValues("hive.default", "simple_json") .baselineValues("hive.default", "partition_with_few_schemas") + .baselineValues("hive.default", "kv_native") + .baselineValues("hive.default", "kv_native_ext") .go(); testBuilder() @@ -249,9 +250,10 @@ public void showInfoSchema() throws Exception { .baselineValues("DRILL", "hive.default", "partition_pruning_test", "TABLE") .baselineValues("DRILL", "hive.default", "partition_with_few_schemas", "TABLE") .baselineValues("DRILL", "hive.default", "kv_parquet", "TABLE") - .baselineValues("DRILL", "hive.default", "countstar_parquet", "TABLE") .baselineValues("DRILL", "hive.default", "kv_sh", "TABLE") .baselineValues("DRILL", "hive.default", "simple_json", "TABLE") + .baselineValues("DRILL", "hive.default", "kv_native", "TABLE") + .baselineValues("DRILL", "hive.default", "kv_native_ext", "TABLE") .baselineValues("DRILL", "hive.skipper", "kv_text_small", "TABLE") .baselineValues("DRILL", "hive.skipper", "kv_text_large", "TABLE") .baselineValues("DRILL", "hive.skipper", "kv_incorrect_skip_header", "TABLE") diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java index 3706ff21fba..64680c03090 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java @@ -35,7 +35,7 @@ public class TestViewSupportOnHiveTables extends TestBaseViewSupport { @BeforeClass public static void generateHive() throws Exception{ - hiveTest = HiveTestDataGenerator.getInstance(dirTestWatcher.getRootDir()); + hiveTest = HiveTestDataGenerator.getInstance(dirTestWatcher); hiveTest.addHiveTestPlugin(getDrillbitContext().getStorage()); } diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java index 78e5b393a39..f2069999849 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.nio.file.Files; +import java.nio.file.Paths; import java.nio.file.attribute.PosixFilePermission; import java.sql.Date; import java.sql.Timestamp; @@ -31,6 +32,7 @@ import com.google.common.io.Resources; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.drill.test.BaseDirTestWatcher; import org.apache.drill.test.BaseTestQuery; import org.apache.drill.common.exceptions.DrillException; import org.apache.drill.exec.store.StoragePluginRegistry; @@ -52,9 +54,11 @@ public class HiveTestDataGenerator { private final String dbDir; private final String whDir; + private final BaseDirTestWatcher dirTestWatcher; private final Map config; - public static synchronized HiveTestDataGenerator getInstance(File baseDir) throws Exception { + public static synchronized HiveTestDataGenerator getInstance(BaseDirTestWatcher dirTestWatcher) throws Exception { + File baseDir = dirTestWatcher.getRootDir(); if (instance == null || !HiveTestDataGenerator.baseDir.equals(baseDir)) { HiveTestDataGenerator.baseDir = baseDir; @@ -64,19 +68,20 @@ public static synchronized HiveTestDataGenerator getInstance(File baseDir) throw final String dbDir = dbDirFile.getAbsolutePath(); final String whDir = whDirFile.getAbsolutePath(); - instance = new HiveTestDataGenerator(dbDir, whDir); + instance = new HiveTestDataGenerator(dbDir, whDir, dirTestWatcher); instance.generateTestData(); } return instance; } - private HiveTestDataGenerator(final String dbDir, final String whDir) { + private HiveTestDataGenerator(final String dbDir, final String whDir, final BaseDirTestWatcher dirTestWatcher) { this.dbDir = dbDir; this.whDir = whDir; + this.dirTestWatcher = dirTestWatcher; config = Maps.newHashMap(); - config.put("hive.metastore.uris", ""); + config.put(ConfVars.METASTOREURIS.toString(), ""); config.put("javax.jdo.option.ConnectionURL", String.format("jdbc:derby:;databaseName=%s;create=true", dbDir)); config.put("hive.metastore.warehouse.dir", whDir); config.put(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS); @@ -128,7 +133,7 @@ public static File createFileWithPermissions(File baseDir, String name) { try { Files.setPosixFilePermissions(dir.toPath(), perms); } catch (IOException e) { - new RuntimeException(e); + throw new RuntimeException(e); } return dir; @@ -494,22 +499,6 @@ private void generateTestData() throws Exception { executeQuery(hiveDriver, "INSERT INTO TABLE kv_parquet PARTITION(part1) SELECT key, value, key FROM default.kv"); executeQuery(hiveDriver, "ALTER TABLE kv_parquet ADD COLUMNS (newcol string)"); - executeQuery(hiveDriver, - "CREATE TABLE countStar_Parquet (int_field INT) STORED AS parquet"); - - final int numOfRows = 200; - final StringBuffer sb = new StringBuffer(); - sb.append("VALUES "); - for(int i = 0; i < numOfRows; ++i) { - if(i != 0) { - sb.append(","); - } - sb.append("(").append(i).append(")"); - } - - executeQuery(hiveDriver, "INSERT INTO TABLE countStar_Parquet \n" + - sb.toString()); - // Create a StorageHandler based table (DRILL-3739) executeQuery(hiveDriver, "CREATE TABLE kv_sh(key INT, value STRING) STORED BY " + "'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'"); @@ -551,9 +540,60 @@ private void generateTestData() throws Exception { Resources.getResource("simple.json") + "' into table default.simple_json"; executeQuery(hiveDriver, loadData); + createTestDataForDrillNativeParquetReaderTests(hiveDriver); + ss.close(); } + private void createTestDataForDrillNativeParquetReaderTests(Driver hiveDriver) { + // Hive managed table that has data qualified for Drill native filter push down + executeQuery(hiveDriver, "create table kv_native(key int, sub_key int) stored as parquet"); + // each insert is created in separate file + executeQuery(hiveDriver, "insert into table kv_native values (1, 1), (1, 2)"); + executeQuery(hiveDriver, "insert into table kv_native values (1, 3), (1, 4)"); + executeQuery(hiveDriver, "insert into table kv_native values (2, 5), (2, 6)"); + executeQuery(hiveDriver, "insert into table kv_native values (null, 9), (null, 10)"); + + // Hive external table which has three partitions + + // copy external table with data from test resources + dirTestWatcher.copyResourceToRoot(Paths.get("external")); + + File external = new File (baseDir, "external"); + String tableLocation = new File(external, "kv_native_ext").toURI().getPath(); + + executeQuery(hiveDriver, String.format("create external table kv_native_ext(key int) " + + "partitioned by (part_key int) " + + "stored as parquet location '%s'", + tableLocation)); + + /* + DATA: + key, part_key + 1, 1 + 2, 1 + 3, 2 + 4, 2 + */ + + // add partitions + + // partition in the same location as table + String firstPartition = new File(tableLocation, "part_key=1").toURI().getPath(); + executeQuery(hiveDriver, String.format("alter table kv_native_ext add partition (part_key = '1') " + + "location '%s'", firstPartition)); + + // partition in different location with table + String secondPartition = new File(external, "part_key=2").toURI().getPath(); + executeQuery(hiveDriver, String.format("alter table kv_native_ext add partition (part_key = '2') " + + "location '%s'", secondPartition)); + + // add empty partition + String thirdPartition = new File(dirTestWatcher.makeSubDir(Paths.get("empty_part")), "part_key=3").toURI().getPath(); + executeQuery(hiveDriver, String.format("alter table kv_native_ext add partition (part_key = '3') " + + "location '%s'", thirdPartition)); + } + private File getTempFile() throws Exception { return java.nio.file.Files.createTempFile("drill-hive-test", ".txt").toFile(); } diff --git a/contrib/storage-hive/core/src/test/resources/external/kv_native_ext/part_key=1/kv_1.parquet b/contrib/storage-hive/core/src/test/resources/external/kv_native_ext/part_key=1/kv_1.parquet new file mode 100755 index 0000000000000000000000000000000000000000..f641402fa92e01e210fdfe40df632604475b7b8b GIT binary patch literal 220 zcmZXP%?iRW5QNt>NWgPS3pw8%Z&BDU5e_SwC9yoL=hSW$LE7U4uAUMsYA!ZFSZ`DwRyN jG?`K}of@r5o2}-FG;xxw?0ji;Vl!LX34E|*j(>dtBnv1S literal 0 HcmV?d00001 diff --git a/contrib/storage-hive/core/src/test/resources/external/part_key=2/kv_2.parquet b/contrib/storage-hive/core/src/test/resources/external/part_key=2/kv_2.parquet new file mode 100755 index 0000000000000000000000000000000000000000..c34c26713fe541b67a89ecd3bc353073d47f9ba4 GIT binary patch literal 220 zcmZXPO$&lR5QYa`MCiPtum>-Rfkl3V3SGL@P5J@GuDgrUx0TS(>7O=s?eMO0ck~&0InNz$GTSg8~C(nqxZ-sNqH1N!tzT@Mv0f=F~toHC*$k-rx|A zc`UFQuC2hby&tB?o!Y4bd;OP=oItp{@&5i&&-uCc2h~5i>&1^3#Ul1Y@#;>}&$!^3 j fileSystems; private ControlsInjector injector; + private boolean allowCreatingFileSystem = true; public BaseOperatorContext(FragmentContext context, BufferAllocator allocator, PhysicalOperator popConfig) { @@ -55,6 +57,7 @@ public BaseOperatorContext(FragmentContext context, BufferAllocator allocator, this.allocator = allocator; this.popConfig = popConfig; this.manager = new BufferManagerImpl(allocator); + this.fileSystems = new ArrayList<>(); } @Override @@ -158,35 +161,46 @@ public void close() { } catch (RuntimeException e) { ex = ex == null ? e : ex; } - try { - if (fs != null) { + + for (DrillFileSystem fs : fileSystems) { + try { fs.close(); - fs = null; - } - } catch (IOException e) { + } catch (IOException e) { throw UserException.resourceError(e) - .addContext("Failed to close the Drill file system for " + getName()) - .build(logger); + .addContext("Failed to close the Drill file system for " + getName()) + .build(logger); + } } + if (ex != null) { throw ex; } } + /** + * Creates DrillFileSystem that automatically tracks operator stats. + * Only one tracking and no non-tracking file system per operator context. + */ @Override public DrillFileSystem newFileSystem(Configuration conf) throws IOException { - Preconditions.checkState(fs == null, "Tried to create a second FileSystem. Can only be called once per OperatorContext"); - fs = new DrillFileSystem(conf, getStats()); + Preconditions.checkState(allowCreatingFileSystem, "Only one tracking file system is allowed per Operator Context and it is already created."); + Preconditions.checkState(fileSystems.isEmpty(), "Non-tracking file system(-s) is(are) already created."); + DrillFileSystem fs = new DrillFileSystem(conf, getStats()); + fileSystems.add(fs); + allowCreatingFileSystem = false; return fs; } /** * Creates a DrillFileSystem that does not automatically track operator stats. + * Multiple non-tracking file system are allowed. */ @Override public DrillFileSystem newNonTrackingFileSystem(Configuration conf) throws IOException { - Preconditions.checkState(fs == null, "Tried to create a second FileSystem. Can only be called once per OperatorContext"); - fs = new DrillFileSystem(conf, null); + Preconditions.checkState(allowCreatingFileSystem, "Only one tracking file system is allowed per Operator Context and it is already created."); + DrillFileSystem fs = new DrillFileSystem(conf, null); + fileSystems.add(fs); return fs; } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java index b2ddf68b959..e89a1f92901 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java @@ -24,12 +24,16 @@ import com.google.common.collect.Lists; +import org.apache.drill.common.expression.LogicalExpression; import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; +import org.apache.drill.exec.ops.UdfUtilities; import org.apache.drill.exec.physical.EndpointAffinity; import org.apache.drill.exec.planner.fragment.DistributionAffinity; import org.apache.drill.exec.planner.physical.PlannerSettings; import com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.drill.exec.server.options.OptionManager; public abstract class AbstractGroupScan extends AbstractBase implements GroupScan { @@ -164,4 +168,14 @@ public Collection getFiles() { public DistributionAffinity getDistributionAffinity() { return DistributionAffinity.SOFT; } + + @Override + public LogicalExpression getFilter() { + return null; + } + + @Override + public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities, FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) { + return null; + } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java index fc63c7763ba..33ab13df16a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java @@ -21,13 +21,17 @@ import java.util.List; import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.common.expression.LogicalExpression; import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; +import org.apache.drill.exec.ops.UdfUtilities; import org.apache.drill.exec.physical.PhysicalOperatorSetupException; import org.apache.drill.exec.planner.physical.PlannerSettings; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.collect.ImmutableList; +import org.apache.drill.exec.server.options.OptionManager; /** * A GroupScan operator represents all data which will be scanned by a given physical @@ -134,4 +138,10 @@ public interface GroupScan extends Scan, HasAffinity{ */ Collection getFiles(); + @JsonIgnore + LogicalExpression getFilter(); + + GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities, + FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager); + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java index 4404a98892c..f2974e16c5f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ScanStats.java @@ -18,7 +18,6 @@ package org.apache.drill.exec.physical.base; public class ScanStats { -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanStats.class); public static final ScanStats TRIVIAL_TABLE = new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, 20, 1, 1); @@ -30,7 +29,6 @@ public class ScanStats { private final GroupScanProperty property; public ScanStats(GroupScanProperty property, long recordCount, float cpuCost, float diskCost) { - super(); this.recordCount = recordCount; this.cpuCost = cpuCost; this.diskCost = diskCost; @@ -49,6 +47,11 @@ public float getDiskCost() { return diskCost; } + @Override + public String toString() { + return "ScanStats{" + "recordCount=" + recordCount + ", cpuCost=" + cpuCost + ", diskCost=" + diskCost + ", property=" + property + '}'; + } + /** * Return if GroupScan knows the exact row count in the result of getSize() call. * By default, groupscan does not know the exact row count, before it scans every rows. @@ -60,7 +63,7 @@ public GroupScanProperty getGroupScanProperty() { - public static enum GroupScanProperty { + public enum GroupScanProperty { NO_EXACT_ROW_COUNT(false, false), EXACT_ROW_COUNT(true, 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 a183e51e3f0..cbc530b40b3 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 @@ -21,7 +21,6 @@ import org.apache.calcite.util.BitSets; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos; -import org.apache.drill.exec.physical.base.FileGroupScan; import org.apache.drill.exec.physical.base.GroupScan; import org.apache.drill.exec.planner.logical.DrillRel; import org.apache.drill.exec.planner.logical.DrillScanRel; @@ -29,31 +28,57 @@ import org.apache.drill.exec.store.dfs.FileSelection; import org.apache.drill.exec.store.dfs.FormatSelection; import org.apache.drill.exec.store.dfs.MetadataContext; -import org.apache.drill.exec.store.parquet.ParquetGroupScan; +import org.apache.drill.exec.store.parquet.AbstractParquetGroupScan; +import org.apache.drill.exec.store.parquet.ParquetReaderUtility; +import org.apache.drill.exec.util.DecimalUtility; +import org.apache.drill.exec.vector.NullableBigIntVector; +import org.apache.drill.exec.vector.NullableBitVector; +import org.apache.drill.exec.vector.NullableDateVector; +import org.apache.drill.exec.vector.NullableDecimal18Vector; +import org.apache.drill.exec.vector.NullableFloat4Vector; +import org.apache.drill.exec.vector.NullableFloat8Vector; +import org.apache.drill.exec.vector.NullableIntVector; +import org.apache.drill.exec.vector.NullableIntervalVector; +import org.apache.drill.exec.vector.NullableSmallIntVector; +import org.apache.drill.exec.vector.NullableTimeStampVector; +import org.apache.drill.exec.vector.NullableTimeVector; +import org.apache.drill.exec.vector.NullableTinyIntVector; +import org.apache.drill.exec.vector.NullableUInt1Vector; +import org.apache.drill.exec.vector.NullableUInt2Vector; +import org.apache.drill.exec.vector.NullableUInt4Vector; +import org.apache.drill.exec.vector.NullableVarBinaryVector; +import org.apache.drill.exec.vector.NullableVarCharVector; import org.apache.drill.exec.vector.ValueVector; import com.google.common.collect.Lists; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.io.api.Binary; +import org.joda.time.DateTimeConstants; import java.io.IOException; +import java.util.ArrayList; import java.util.BitSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; - /** * PartitionDescriptor that describes partitions based on column names instead of directory structure */ public class ParquetPartitionDescriptor extends AbstractPartitionDescriptor { - private final List partitionColumns; + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetPartitionDescriptor.class); + private final DrillScanRel scanRel; + private final AbstractParquetGroupScan groupScan; + private final List partitionColumns; public ParquetPartitionDescriptor(PlannerSettings settings, DrillScanRel scanRel) { - ParquetGroupScan scan = (ParquetGroupScan) scanRel.getGroupScan(); - this.partitionColumns = scan.getPartitionColumns(); this.scanRel = scanRel; + assert scanRel.getGroupScan() instanceof AbstractParquetGroupScan; + this.groupScan = (AbstractParquetGroupScan) scanRel.getGroupScan(); + this.partitionColumns = groupScan.getPartitionColumns(); } @Override @@ -81,15 +106,6 @@ public int getMaxHierarchyLevel() { return partitionColumns.size(); } - private GroupScan createNewGroupScan(List newFiles, String cacheFileRoot, - boolean wasAllPartitionsPruned, MetadataContext metaContext) throws IOException { - final FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation(), - cacheFileRoot, wasAllPartitionsPruned); - newSelection.setMetaContext(metaContext); - final FileGroupScan newScan = ((FileGroupScan)scanRel.getGroupScan()).clone(newSelection); - return newScan; - } - @Override public void populatePartitionVectors(ValueVector[] vectors, List partitions, BitSet partitionColumnBitSet, Map fieldNameMap) { @@ -97,8 +113,7 @@ int record = 0; for (PartitionLocation partitionLocation: partitions) { for (int partitionColumnIndex : BitSets.toIter(partitionColumnBitSet)) { SchemaPath column = SchemaPath.getSimplePath(fieldNameMap.get(partitionColumnIndex)); - ((ParquetGroupScan) scanRel.getGroupScan()).populatePruningVector(vectors[partitionColumnIndex], record, column, - partitionLocation.getEntirePartitionLocation()); + populatePruningVector(vectors[partitionColumnIndex], record, column, partitionLocation.getEntirePartitionLocation()); } record++; } @@ -114,7 +129,7 @@ int record = 0; @Override public TypeProtos.MajorType getVectorType(SchemaPath column, PlannerSettings plannerSettings) { - return ((ParquetGroupScan) scanRel.getGroupScan()).getTypeForColumn(column); + return groupScan.getTypeForColumn(column); } @Override @@ -124,26 +139,22 @@ public String getBaseTableLocation() { } @Override - protected void createPartitionSublists() { - Set fileLocations = ((ParquetGroupScan) scanRel.getGroupScan()).getFileSet(); - List locations = new LinkedList<>(); - for (String file: fileLocations) { - locations.add(new ParquetPartitionLocation(file)); - } - locationSuperList = Lists.partition(locations, PartitionDescriptor.PARTITION_BATCH_SIZE); - sublistsCreated = true; - } - - @Override - public TableScan createTableScan(List newPartitionLocation, String cacheFileRoot, - boolean wasAllPartitionsPruned, MetadataContext metaContext) throws Exception { - List newFiles = Lists.newArrayList(); + public TableScan createTableScan(List newPartitionLocation, + String cacheFileRoot, + boolean wasAllPartitionsPruned, + MetadataContext metaContext) throws Exception { + List newFiles = new ArrayList<>(); for (final PartitionLocation location : newPartitionLocation) { newFiles.add(location.getEntirePartitionLocation()); } final GroupScan newGroupScan = createNewGroupScan(newFiles, cacheFileRoot, wasAllPartitionsPruned, metaContext); + if (newGroupScan == null) { + logger.warn("Unable to create new group scan, returning original table scan."); + return scanRel; + } + return new DrillScanRel(scanRel.getCluster(), scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL), scanRel.getTable(), @@ -154,9 +165,261 @@ public TableScan createTableScan(List newPartitionLocation, S } @Override - public TableScan createTableScan(List newPartitionLocation, - boolean wasAllPartitionsPruned) throws Exception { + public TableScan createTableScan(List newPartitionLocation, boolean wasAllPartitionsPruned) throws Exception { return createTableScan(newPartitionLocation, null, wasAllPartitionsPruned, null); } + @Override + protected void createPartitionSublists() { + Set fileLocations = groupScan.getFileSet(); + List locations = new LinkedList<>(); + for (String file : fileLocations) { + locations.add(new ParquetPartitionLocation(file)); + } + locationSuperList = Lists.partition(locations, PartitionDescriptor.PARTITION_BATCH_SIZE); + sublistsCreated = true; + } + + private GroupScan createNewGroupScan(List newFiles, + String cacheFileRoot, + boolean wasAllPartitionsPruned, + MetadataContext metaContext) throws IOException { + + FileSelection newSelection = FileSelection.create(null, newFiles, getBaseTableLocation(), cacheFileRoot, wasAllPartitionsPruned); + if (newSelection == null) { + return null; + } + newSelection.setMetaContext(metaContext); + return groupScan.clone(newSelection); + } + + private void populatePruningVector(ValueVector v, int index, SchemaPath column, String file) { + String path = Path.getPathWithoutSchemeAndAuthority(new Path(file)).toString(); + TypeProtos.MajorType majorType = getVectorType(column, null); + TypeProtos.MinorType type = majorType.getMinorType(); + switch (type) { + case BIT: { + NullableBitVector bitVector = (NullableBitVector) v; + Boolean value = groupScan.getPartitionValue(path, column, Boolean.class); + if (value == null) { + bitVector.getMutator().setNull(index); + } else { + bitVector.getMutator().setSafe(index, value ? 1 : 0); + } + return; + } + case INT: { + NullableIntVector intVector = (NullableIntVector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + intVector.getMutator().setNull(index); + } else { + intVector.getMutator().setSafe(index, value); + } + return; + } + case SMALLINT: { + NullableSmallIntVector smallIntVector = (NullableSmallIntVector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + smallIntVector.getMutator().setNull(index); + } else { + smallIntVector.getMutator().setSafe(index, value.shortValue()); + } + return; + } + case TINYINT: { + NullableTinyIntVector tinyIntVector = (NullableTinyIntVector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + tinyIntVector.getMutator().setNull(index); + } else { + tinyIntVector.getMutator().setSafe(index, value.byteValue()); + } + return; + } + case UINT1: { + NullableUInt1Vector intVector = (NullableUInt1Vector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + intVector.getMutator().setNull(index); + } else { + intVector.getMutator().setSafe(index, value.byteValue()); + } + return; + } + case UINT2: { + NullableUInt2Vector intVector = (NullableUInt2Vector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + intVector.getMutator().setNull(index); + } else { + intVector.getMutator().setSafe(index, (char) value.shortValue()); + } + return; + } + case UINT4: { + NullableUInt4Vector intVector = (NullableUInt4Vector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + intVector.getMutator().setNull(index); + } else { + intVector.getMutator().setSafe(index, value); + } + return; + } + case BIGINT: { + NullableBigIntVector bigIntVector = (NullableBigIntVector) v; + Long value = groupScan.getPartitionValue(path, column, Long.class); + if (value == null) { + bigIntVector.getMutator().setNull(index); + } else { + bigIntVector.getMutator().setSafe(index, value); + } + return; + } + case FLOAT4: { + NullableFloat4Vector float4Vector = (NullableFloat4Vector) v; + Float value = groupScan.getPartitionValue(path, column, Float.class); + if (value == null) { + float4Vector.getMutator().setNull(index); + } else { + float4Vector.getMutator().setSafe(index, value); + } + return; + } + case FLOAT8: { + NullableFloat8Vector float8Vector = (NullableFloat8Vector) v; + Double value = groupScan.getPartitionValue(path, column, Double.class); + if (value == null) { + float8Vector.getMutator().setNull(index); + } else { + float8Vector.getMutator().setSafe(index, value); + } + return; + } + case VARBINARY: { + NullableVarBinaryVector varBinaryVector = (NullableVarBinaryVector) v; + Object s = groupScan.getPartitionValue(path, column, Object.class); + byte[] bytes; + if (s == null) { + varBinaryVector.getMutator().setNull(index); + return; + } else { + bytes = getBytes(type, s); + } + varBinaryVector.getMutator().setSafe(index, bytes, 0, bytes.length); + return; + } + case DECIMAL18: { + NullableDecimal18Vector decimalVector = (NullableDecimal18Vector) v; + Object s = groupScan.getPartitionValue(path, column, Object.class); + byte[] bytes; + if (s == null) { + decimalVector.getMutator().setNull(index); + return; + } else if (s instanceof Integer) { + long value = DecimalUtility.getBigDecimalFromPrimitiveTypes( + (Integer) s, + majorType.getScale(), + majorType.getPrecision()).longValue(); + decimalVector.getMutator().setSafe(index, value); + return; + } else if (s instanceof Long) { + long value = DecimalUtility.getBigDecimalFromPrimitiveTypes( + (Long) s, + majorType.getScale(), + majorType.getPrecision()).longValue(); + decimalVector.getMutator().setSafe(index, value); + return; + } else { + bytes = getBytes(type, s); + } + long value = DecimalUtility.getBigDecimalFromByteArray(bytes, 0, bytes.length, majorType.getScale()).longValue(); + decimalVector.getMutator().setSafe(index, value); + return; + } + case DATE: { + NullableDateVector dateVector = (NullableDateVector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + dateVector.getMutator().setNull(index); + } else { + dateVector.getMutator().setSafe(index, value * (long) DateTimeConstants.MILLIS_PER_DAY); + } + return; + } + case TIME: { + NullableTimeVector timeVector = (NullableTimeVector) v; + Integer value = groupScan.getPartitionValue(path, column, Integer.class); + if (value == null) { + timeVector.getMutator().setNull(index); + } else { + timeVector.getMutator().setSafe(index, value); + } + return; + } + case TIMESTAMP: { + NullableTimeStampVector timeStampVector = (NullableTimeStampVector) v; + Long value = groupScan.getPartitionValue(path, column, Long.class); + if (value == null) { + timeStampVector.getMutator().setNull(index); + } else { + timeStampVector.getMutator().setSafe(index, value); + } + return; + } + case VARCHAR: { + NullableVarCharVector varCharVector = (NullableVarCharVector) v; + Object s = groupScan.getPartitionValue(path, column, Object.class); + byte[] bytes; + if (s == null) { + varCharVector.getMutator().setNull(index); + return; + } else { + bytes = getBytes(type, s); + } + varCharVector.getMutator().setSafe(index, bytes, 0, bytes.length); + return; + } + case INTERVAL: { + NullableIntervalVector intervalVector = (NullableIntervalVector) v; + Object s = groupScan.getPartitionValue(path, column, Object.class); + byte[] bytes; + if (s == null) { + intervalVector.getMutator().setNull(index); + return; + } else { + bytes = getBytes(type, s); + } + intervalVector.getMutator().setSafe(index, 1, + ParquetReaderUtility.getIntFromLEBytes(bytes, 0), + ParquetReaderUtility.getIntFromLEBytes(bytes, 4), + ParquetReaderUtility.getIntFromLEBytes(bytes, 8)); + return; + } + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + /** + * Returns the sequence of bytes received from {@code Object source}. + * + * @param type the column type + * @param source the source of the bytes sequence + * @return bytes sequence obtained from {@code Object source} + */ + private byte[] getBytes(TypeProtos.MinorType type, Object source) { + byte[] bytes; + if (source instanceof Binary) { + bytes = ((Binary) source).getBytes(); + } else if (source instanceof byte[]) { + bytes = (byte[]) source; + } else { + throw new UnsupportedOperationException("Unable to create column data for type: " + type); + } + return bytes; + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java index f71c2815366..96cfa8aa866 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java @@ -365,6 +365,7 @@ static RuleSet getPhysicalPruneScanRules(OptimizerRulesContext optimizerRulesCon // Ideally this should be done in logical planning, before join order planning is done. // Before we can make such change, we have to figure out how to adjust the selectivity // estimation of filter operator, after filter is pushed down to scan. + ParquetPushDownFilter.getFilterOnProject(optimizerRulesContext), ParquetPushDownFilter.getFilterOnScan(optimizerRulesContext) ) @@ -426,6 +427,9 @@ static RuleSet getPhysicalRules(OptimizerRulesContext optimizerRulesContext) { ruleList.add(ValuesPrule.INSTANCE); ruleList.add(DirectScanPrule.INSTANCE); + ruleList.add(DrillPushLimitToScanRule.LIMIT_ON_PROJECT); + ruleList.add(DrillPushLimitToScanRule.LIMIT_ON_SCAN); + if (ps.isHashAggEnabled()) { ruleList.add(HashAggPrule.INSTANCE); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterItemStarReWriterRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterItemStarReWriterRule.java index 27f8c49c6c1..b7cdcfc9b6b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterItemStarReWriterRule.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterItemStarReWriterRule.java @@ -32,7 +32,7 @@ import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl; import org.apache.drill.exec.planner.types.RelDataTypeHolder; -import org.apache.drill.exec.store.parquet.ParquetGroupScan; +import org.apache.drill.exec.store.parquet.AbstractParquetGroupScan; import java.util.ArrayList; import java.util.Collection; @@ -73,7 +73,7 @@ private static class ProjectOnScan extends RelOptRule { @Override public boolean matches(RelOptRuleCall call) { DrillScanRel scan = call.rel(1); - return scan.getGroupScan() instanceof ParquetGroupScan && super.matches(call); + return scan.getGroupScan() instanceof AbstractParquetGroupScan && super.matches(call); } @Override @@ -129,7 +129,7 @@ private static class FilterOnScan extends RelOptRule { @Override public boolean matches(RelOptRuleCall call) { DrillScanRel scan = call.rel(1); - return scan.getGroupScan() instanceof ParquetGroupScan && super.matches(call); + return scan.getGroupScan() instanceof AbstractParquetGroupScan && super.matches(call); } @Override @@ -149,7 +149,7 @@ private static class FilterProjectScan extends RelOptRule { @Override public boolean matches(RelOptRuleCall call) { DrillScanRel scan = call.rel(2); - return scan.getGroupScan() instanceof ParquetGroupScan && super.matches(call); + return scan.getGroupScan() instanceof AbstractParquetGroupScan && super.matches(call); } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/ParquetPruneScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/ParquetPruneScanRule.java index 3153b9d4859..6e44383c018 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/ParquetPruneScanRule.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/ParquetPruneScanRule.java @@ -19,10 +19,8 @@ import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.TableScan; import org.apache.drill.exec.ops.OptimizerRulesContext; -import org.apache.drill.exec.physical.base.FileGroupScan; import org.apache.drill.exec.physical.base.GroupScan; import org.apache.drill.exec.planner.ParquetPartitionDescriptor; import org.apache.drill.exec.planner.PartitionDescriptor; @@ -32,11 +30,11 @@ import org.apache.drill.exec.planner.logical.RelOptHelper; import org.apache.drill.exec.planner.physical.PlannerSettings; import org.apache.drill.exec.planner.physical.PrelUtil; -import org.apache.drill.exec.store.parquet.ParquetGroupScan; +import org.apache.drill.exec.store.parquet.AbstractParquetGroupScan; public class ParquetPruneScanRule { - public static final RelOptRule getFilterOnProjectParquet(OptimizerRulesContext optimizerRulesContext) { + public static RelOptRule getFilterOnProjectParquet(OptimizerRulesContext optimizerRulesContext) { return new PruneScanRule( RelOptHelper.some(DrillFilterRel.class, RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class))), "PruneScanRule:Filter_On_Project_Parquet", @@ -53,9 +51,9 @@ public boolean matches(RelOptRuleCall call) { GroupScan groupScan = scan.getGroupScan(); // this rule is applicable only for parquet based partition pruning if (PrelUtil.getPlannerSettings(scan.getCluster().getPlanner()).isHepPartitionPruningEnabled()) { - return groupScan instanceof ParquetGroupScan && groupScan.supportsPartitionFilterPushdown() && !scan.partitionFilterPushdown(); + return groupScan instanceof AbstractParquetGroupScan && groupScan.supportsPartitionFilterPushdown() && !scan.partitionFilterPushdown(); } else { - return groupScan instanceof ParquetGroupScan && groupScan.supportsPartitionFilterPushdown(); + return groupScan instanceof AbstractParquetGroupScan && groupScan.supportsPartitionFilterPushdown(); } } @@ -69,7 +67,7 @@ public void onMatch(RelOptRuleCall call) { }; } - public static final RelOptRule getFilterOnScanParquet(OptimizerRulesContext optimizerRulesContext) { + public static RelOptRule getFilterOnScanParquet(OptimizerRulesContext optimizerRulesContext) { return new PruneScanRule( RelOptHelper.some(DrillFilterRel.class, RelOptHelper.any(DrillScanRel.class)), "PruneScanRule:Filter_On_Scan_Parquet", optimizerRulesContext) { @@ -85,9 +83,9 @@ public boolean matches(RelOptRuleCall call) { GroupScan groupScan = scan.getGroupScan(); // this rule is applicable only for parquet based partition pruning if (PrelUtil.getPlannerSettings(scan.getCluster().getPlanner()).isHepPartitionPruningEnabled()) { - return groupScan instanceof ParquetGroupScan && groupScan.supportsPartitionFilterPushdown() && !scan.partitionFilterPushdown(); + return groupScan instanceof AbstractParquetGroupScan && groupScan.supportsPartitionFilterPushdown() && !scan.partitionFilterPushdown(); } else { - return groupScan instanceof ParquetGroupScan && groupScan.supportsPartitionFilterPushdown(); + return groupScan instanceof AbstractParquetGroupScan && groupScan.supportsPartitionFilterPushdown(); } } 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 5f679a4e67e..7fa17940bcf 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 @@ -24,7 +24,6 @@ import java.util.concurrent.TimeUnit; import com.google.common.base.Stopwatch; - import org.apache.calcite.adapter.enumerable.EnumerableTableScan; import org.apache.calcite.rel.core.Filter; import org.apache.calcite.rel.core.Project; @@ -147,8 +146,8 @@ public static RelOptRule getDirFilterOnScan(OptimizerRulesContext optimizerRules protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectRel, TableScan scanRel) { final String pruningClassName = getClass().getName(); - logger.info("Beginning partition pruning, pruning class: {}", pruningClassName); - Stopwatch totalPruningTime = Stopwatch.createStarted(); + logger.debug("Beginning partition pruning, pruning class: {}", pruningClassName); + Stopwatch totalPruningTime = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; final PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner()); PartitionDescriptor descriptor = getPartitionDescriptor(settings, scanRel); @@ -191,30 +190,33 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR } if (partitionColumnBitSet.isEmpty()) { - logger.info("No partition columns are projected from the scan..continue. " + - "Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + if (totalPruningTime != null) { + logger.debug("No partition columns are projected from the scan..continue. Total pruning elapsed time: {} ms", + totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + } setPruneStatus(metaContext, PruneStatus.NOT_PRUNED); return; } // stop watch to track how long we spend in different phases of pruning - Stopwatch miscTimer = Stopwatch.createUnstarted(); - - // track how long we spend building the filter tree - miscTimer.start(); + // first track how long we spend building the filter tree + Stopwatch miscTimer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; 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)); - miscTimer.reset(); + if (miscTimer != null) { + logger.debug("Total elapsed time to build and analyze filter tree: {} ms", miscTimer.elapsed(TimeUnit.MILLISECONDS)); + miscTimer.reset(); + } if (pruneCondition == null) { - logger.info("No conditions were found eligible for partition pruning." + - "Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + if (totalPruningTime != null) { + logger.debug("No conditions were found eligible for partition pruning. Total pruning elapsed time: {} ms", + totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + } setPruneStatus(metaContext, PruneStatus.NOT_PRUNED); return; } @@ -251,15 +253,19 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR container.add(v); } - // track how long we spend populating partition column vectors - miscTimer.start(); + if (miscTimer != null) { + // track how long we spend populating partition column vectors + miscTimer.start(); + } // populate partition vectors. descriptor.populatePartitionVectors(vectors, partitions, partitionColumnBitSet, fieldNameMap); - logger.info("Elapsed time to populate partitioning column vectors: {} ms within batchIndex: {}", - miscTimer.elapsed(TimeUnit.MILLISECONDS), batchIndex); - miscTimer.reset(); + if (miscTimer != null) { + logger.debug("Elapsed time to populate partitioning column vectors: {} ms within batchIndex: {}", + miscTimer.elapsed(TimeUnit.MILLISECONDS), batchIndex); + miscTimer.reset(); + } // materialize the expression; only need to do this once if (batchIndex == 0) { @@ -267,8 +273,9 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR if (materializedExpr == null) { // continue without partition pruning; no need to log anything here since // materializePruneExpr logs it already - logger.info("Total pruning elapsed time: {} ms", - totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + if (totalPruningTime != null) { + logger.debug("Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + } setPruneStatus(metaContext, PruneStatus.NOT_PRUNED); return; } @@ -276,14 +283,18 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR output.allocateNew(partitions.size()); - // start the timer to evaluate how long we spend in the interpreter evaluation - miscTimer.start(); + if (miscTimer != null) { + // start the timer to evaluate how long we spend in the interpreter evaluation + miscTimer.start(); + } InterpreterEvaluator.evaluate(partitions.size(), optimizerContext, container, output, materializedExpr); - logger.info("Elapsed time in interpreter evaluation: {} ms within batchIndex: {} with # of partitions : {}", - miscTimer.elapsed(TimeUnit.MILLISECONDS), batchIndex, partitions.size()); - miscTimer.reset(); + if (miscTimer != null) { + logger.debug("Elapsed time in interpreter evaluation: {} ms within batchIndex: {} with # of partitions : {}", + miscTimer.elapsed(TimeUnit.MILLISECONDS), batchIndex, partitions.size()); + miscTimer.reset(); + } int recordCount = 0; int qualifiedCount = 0; @@ -338,7 +349,9 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR batchIndex++; } catch (Exception e) { logger.warn("Exception while trying to prune partition.", e); - logger.info("Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + if (totalPruningTime != null) { + logger.debug("Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + } setPruneStatus(metaContext, PruneStatus.NOT_PRUNED); return; // continue without partition pruning @@ -352,7 +365,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR try { if (newPartitions.size() == numTotal) { - logger.info("No partitions were eligible for pruning"); + logger.debug("No partitions were eligible for pruning"); return; } @@ -371,7 +384,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR // 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); + logger.debug("All {} partitions were pruned; added back a single partition to allow creating a schema", numTotal); // set the cacheFileRoot appropriately if (firstLocation.isCompositePartition()) { @@ -379,7 +392,7 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR } } - logger.info("Pruned {} partitions down to {}", numTotal, newPartitions.size()); + logger.debug("Pruned {} partitions down to {}", numTotal, newPartitions.size()); List conjuncts = RelOptUtil.conjunctions(condition); List pruneConjuncts = RelOptUtil.conjunctions(pruneCondition); @@ -439,7 +452,9 @@ protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project projectR } catch (Exception e) { logger.warn("Exception while using the pruned partitions.", e); } finally { - logger.info("Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + if (totalPruningTime != null) { + logger.debug("Total pruning elapsed time: {} ms", totalPruningTime.elapsed(TimeUnit.MILLISECONDS)); + } } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java index 6bfceb42aea..f463e6ddc0f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java @@ -37,7 +37,7 @@ import org.apache.drill.exec.store.dfs.FileSystemPlugin; import org.apache.drill.exec.store.dfs.FormatSelection; import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig; -import org.apache.drill.exec.store.parquet.Metadata; +import org.apache.drill.exec.store.parquet.metadata.Metadata; import org.apache.drill.exec.store.parquet.ParquetFormatConfig; import org.apache.drill.exec.work.foreman.ForemanSetupException; import org.apache.hadoop.fs.Path; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java index fb86783c269..5ab67f1ad43 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java @@ -17,24 +17,25 @@ */ package org.apache.drill.exec.store; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.lang3.ArrayUtils; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.map.CaseInsensitiveMap; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.server.options.OptionManager; import org.apache.drill.exec.server.options.OptionValue; import org.apache.drill.exec.store.dfs.FileSelection; -import org.apache.drill.exec.store.dfs.easy.FileWork; import org.apache.drill.exec.util.Utilities; import org.apache.hadoop.fs.Path; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.io.Files; public class ColumnExplorer { @@ -156,43 +157,74 @@ public static List getPartitionColumnNames(FileSelection selection, Sche } /** - * Compares selection root and actual file path to determine partition columns values. - * Adds implicit file columns according to columns list. + * Creates map with implicit columns where key is column name, value is columns actual value. + * This map contains partition and implicit file columns (if requested). + * Partition columns names are formed based in partition designator and value index. * - * @return map with columns names as keys and their values + * @param filePath file path, used to populate file implicit columns + * @param partitionValues list of partition values + * @param includeFileImplicitColumns if file implicit columns should be included into the result + * @return implicit columns map */ - public Map populateImplicitColumns(FileWork work, String selectionRoot) { - return populateImplicitColumns(work.getPath(), selectionRoot); - } + public Map populateImplicitColumns(String filePath, + List partitionValues, + boolean includeFileImplicitColumns) { + Map implicitValues = new LinkedHashMap<>(); - /** - * Compares selection root and actual file path to determine partition columns values. - * Adds implicit file columns according to columns list. - * - * @return map with columns names as keys and their values - */ - public Map populateImplicitColumns(String filePath, String selectionRoot) { - Map implicitValues = Maps.newLinkedHashMap(); - if (selectionRoot != null) { - String[] r = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString().split("/"); - Path path = Path.getPathWithoutSchemeAndAuthority(new Path(filePath)); - String[] p = path.toString().split("/"); - if (p.length > r.length) { - String[] q = ArrayUtils.subarray(p, r.length, p.length - 1); - for (int a = 0; a < q.length; a++) { - if (isStarQuery || selectedPartitionColumns.contains(a)) { - implicitValues.put(partitionDesignator + a, q[a]); - } - } + for (int i = 0; i < partitionValues.size(); i++) { + if (isStarQuery || selectedPartitionColumns.contains(i)) { + implicitValues.put(partitionDesignator + i, partitionValues.get(i)); } - //add implicit file columns + } + + if (includeFileImplicitColumns) { + Path path = Path.getPathWithoutSchemeAndAuthority(new Path(filePath)); for (Map.Entry entry : selectedImplicitColumns.entrySet()) { implicitValues.put(entry.getKey(), entry.getValue().getValue(path)); } } + return implicitValues; } + /** + * Compares root and file path to determine directories + * that are present in the file path but absent in root. + * Example: root - a/b/c, filePath - a/b/c/d/e/0_0_0.parquet, result - d/e. + * Stores different directory names in the list in successive order. + * + * + * @param filePath file path + * @param root root directory + * @return list of directory names + */ + public static List listPartitionValues(String filePath, String root) { + if (filePath == null || root == null) { + return Collections.emptyList(); + } + + int rootDepth = new Path(root).depth(); + Path path = new Path(filePath); + int parentDepth = path.getParent().depth(); + + int diffCount = parentDepth - rootDepth; + + if (diffCount < 1) { + return Collections.emptyList(); + } + + String[] diffDirectoryNames = new String[diffCount]; + + // start filling in array from the end + for (int i = rootDepth; parentDepth > i; i++) { + path = path.getParent(); + // place in the end of array + diffDirectoryNames[parentDepth - i - 1] = path.getName(); + } + + return Arrays.asList(diffDirectoryNames); + } + public boolean isStarQuery() { return isStarQuery; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java index fe0cae19a50..7cce2ad5cc8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java @@ -20,18 +20,15 @@ import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import com.google.common.base.Stopwatch; import org.apache.drill.common.concurrent.ExtendedLatch; import org.apache.drill.common.exceptions.UserException; import org.slf4j.Logger; -import com.google.common.base.Stopwatch; import com.google.common.collect.Lists; /** @@ -115,7 +112,7 @@ public void run() { * @throws IOException All exceptions are coerced to IOException since this was build for storage system tasks initially. */ public static List run(final String activity, final Logger logger, final List> runnables, int parallelism) throws IOException { - Stopwatch watch = Stopwatch.createStarted(); + Stopwatch watch = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; long timedRunnableStart=System.nanoTime(); if(runnables.size() == 1){ parallelism = 1; @@ -186,21 +183,22 @@ public static List run(final String activity, final Logger logger, final } } - if(logger.isInfoEnabled()){ + if (watch != null) { double avg = (sum/1000.0/1000.0)/(count*1.0d); double avgStart = (totalStart/1000.0)/(count*1.0d); - logger.info( + logger.debug( String.format("%s: Executed %d out of %d using %d threads. " + "Time: %dms total, %fms avg, %dms max.", activity, count, runnables.size(), parallelism, watch.elapsed(TimeUnit.MILLISECONDS), avg, max/1000/1000)); - logger.info( + logger.debug( String.format("%s: Executed %d out of %d using %d threads. " + "Earliest start: %f \u03BCs, Latest start: %f \u03BCs, Average start: %f \u03BCs .", activity, count, runnables.size(), parallelism, earliestStart/1000.0, latestStart/1000.0, avgStart)); + watch.stop(); } - if(excep != null) { + if (excep != null) { throw excep; } 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 7edb327b109..f5bcced4eb0 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 @@ -95,7 +95,7 @@ public FileSelection(final List statuses, final List files, final String cacheFileRoot, final boolean wasAllPartitionsPruned, final StatusType dirStatus) { this.statuses = statuses; this.files = files; - this.selectionRoot = Preconditions.checkNotNull(selectionRoot); + this.selectionRoot = selectionRoot; this.dirStatus = dirStatus; this.cacheFileRoot = cacheFileRoot; this.wasAllPartitionsPruned = wasAllPartitionsPruned; @@ -121,7 +121,7 @@ public String getSelectionRoot() { } public List getStatuses(final DrillFileSystem fs) throws IOException { - Stopwatch timer = Stopwatch.createStarted(); + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; if (statuses == null) { final List newStatuses = Lists.newArrayList(); @@ -130,8 +130,11 @@ public List getStatuses(final DrillFileSystem fs) throws IOException } statuses = newStatuses; } - logger.info("FileSelection.getStatuses() took {} ms, numFiles: {}", - timer.elapsed(TimeUnit.MILLISECONDS), statuses == null ? 0 : statuses.size()); + if (timer != null) { + logger.debug("FileSelection.getStatuses() took {} ms, numFiles: {}", + timer.elapsed(TimeUnit.MILLISECONDS), statuses == null ? 0 : statuses.size()); + timer.stop(); + } return statuses; } @@ -164,7 +167,7 @@ public FileSelection minusDirectories(DrillFileSystem fs) throws IOException { if (isExpandedFully()) { return this; } - Stopwatch timer = Stopwatch.createStarted(); + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; List statuses = getStatuses(fs); List nonDirectories = Lists.newArrayList(); @@ -173,8 +176,10 @@ public FileSelection minusDirectories(DrillFileSystem fs) throws IOException { } final FileSelection fileSel = create(nonDirectories, null, selectionRoot); - logger.debug("FileSelection.minusDirectories() took {} ms, numFiles: {}", - timer.elapsed(TimeUnit.MILLISECONDS), statuses.size()); + if (timer != null) { + logger.debug("FileSelection.minusDirectories() took {} ms, numFiles: {}", timer.elapsed(TimeUnit.MILLISECONDS), statuses.size()); + timer.stop(); + } // fileSel will be null if we query an empty folder if (fileSel != null) { @@ -259,7 +264,7 @@ private static String buildPath(final String[] path, final int folderIndex) { public static FileSelection create(final DrillFileSystem fs, final String parent, final String path, final boolean allowAccessOutsideWorkspace) throws IOException { - Stopwatch timer = Stopwatch.createStarted(); + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; boolean hasWildcard = path.contains(WILD_CARD); final Path combined = new Path(parent, removeLeadingSlash(path)); @@ -271,7 +276,10 @@ public static FileSelection create(final DrillFileSystem fs, final String parent return null; } final FileSelection fileSel = create(Lists.newArrayList(statuses), null, combined.toUri().getPath()); - logger.debug("FileSelection.create() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS)); + if (timer != null) { + logger.debug("FileSelection.create() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + } if (fileSel == null) { return null; } @@ -322,7 +330,7 @@ public static FileSelection create(final List statuses, final List dirPaths, final FileSelection selection, final String cacheFileRoot) { - Stopwatch timer = Stopwatch.createStarted(); + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; final String root = selection.getSelectionRoot(); if (Strings.isNullOrEmpty(root)) { throw new DrillRuntimeException("Selection root is null or empty" + root); @@ -338,9 +346,7 @@ public static FileSelection createFromDirectories(final List dirPaths, f dirs.add(status.getPath().toString()); } } else { - for (String s : dirPaths) { - dirs.add(s); - } + dirs.addAll(dirPaths); } final Path rootPath = handleWildCard(root); @@ -349,7 +355,10 @@ public static FileSelection createFromDirectories(final List dirPaths, f final Path path = new Path(uri.getScheme(), uri.getAuthority(), rootPath.toUri().getPath()); FileSelection fileSel = new FileSelection(null, dirs, path.toString(), cacheFileRoot, false); fileSel.setHadWildcard(selection.hadWildcard()); - logger.info("FileSelection.createFromDirectories() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS)); + if (timer != null) { + logger.debug("FileSelection.createFromDirectories() took {} ms ", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + } return fileSel; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java index e02d8417032..15107ac18bf 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/ReadEntryFromHDFS.java @@ -22,14 +22,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -public class ReadEntryFromHDFS extends ReadEntryWithPath implements FileWork{ +public class ReadEntryFromHDFS extends ReadEntryWithPath implements FileWork { private long start; private long length; @JsonCreator public ReadEntryFromHDFS(@JsonProperty("path") String path,@JsonProperty("start") long start, @JsonProperty("length") long length) { - this.path = path; + super(path); this.start = start; this.length = length; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java index 678569f3d53..5af109143c1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java @@ -147,10 +147,12 @@ CloseableRecordBatch getReaderBatch(FragmentContext context, EasySubScan scan) t List readers = new LinkedList<>(); List> implicitColumns = Lists.newArrayList(); Map mapWithMaxColumns = Maps.newLinkedHashMap(); - for(FileWork work : scan.getWorkUnits()){ + boolean supportsFileImplicitColumns = scan.getSelectionRoot() != null; + for (FileWork work : scan.getWorkUnits()){ RecordReader recordReader = getRecordReader(context, dfs, work, scan.getColumns(), scan.getUserName()); readers.add(recordReader); - Map implicitValues = columnExplorer.populateImplicitColumns(work, scan.getSelectionRoot()); + List partitionValues = ColumnExplorer.listPartitionValues(work.getPath(), scan.getSelectionRoot()); + Map implicitValues = columnExplorer.populateImplicitColumns(work.getPath(), partitionValues, supportsFileImplicitColumns); implicitColumns.add(implicitValues); if (implicitValues.size() > mapWithMaxColumns.size()) { mapWithMaxColumns = implicitValues; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java index f1da0f71322..aa3f4aec525 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java @@ -70,7 +70,7 @@ public InfoSchemaTableType getTable() { } @JsonProperty("filter") - public InfoSchemaFilter getFilter() { + public InfoSchemaFilter getSchemaFilter() { return filter; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java new file mode 100644 index 00000000000..1f8c535a912 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java @@ -0,0 +1,463 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; +import org.apache.drill.common.expression.ErrorCollector; +import org.apache.drill.common.expression.ErrorCollectorImpl; +import org.apache.drill.common.expression.ExpressionStringBuilder; +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.common.expression.ValueExpressions; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.exec.compile.sig.ConstantExpressionIdentifier; +import org.apache.drill.exec.expr.ExpressionTreeMaterializer; +import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; +import org.apache.drill.exec.expr.stat.ParquetFilterPredicate; +import org.apache.drill.exec.ops.UdfUtilities; +import org.apache.drill.exec.physical.EndpointAffinity; +import org.apache.drill.exec.physical.base.AbstractFileGroupScan; +import org.apache.drill.exec.physical.base.GroupScan; +import org.apache.drill.exec.physical.base.ScanStats; +import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty; +import org.apache.drill.exec.planner.physical.PlannerSettings; +import org.apache.drill.exec.proto.CoordinationProtos; +import org.apache.drill.exec.server.options.OptionManager; +import org.apache.drill.exec.store.ColumnExplorer; +import org.apache.drill.exec.store.dfs.FileSelection; +import org.apache.drill.exec.store.dfs.ReadEntryWithPath; +import org.apache.drill.exec.store.parquet.stat.ColumnStatistics; +import org.apache.drill.exec.store.parquet.stat.ParquetMetaStatCollector; +import org.apache.drill.exec.store.schedule.AffinityCreator; +import org.apache.drill.exec.store.schedule.AssignmentCreator; +import org.apache.drill.exec.store.schedule.EndpointByteMap; +import org.apache.drill.exec.store.schedule.EndpointByteMapImpl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + + +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.RowGroupMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; + +public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan { + + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractParquetGroupScan.class); + + protected List columns; + protected List entries; + protected LogicalExpression filter; + + protected ParquetTableMetadataBase parquetTableMetadata; + protected List rowGroupInfos; + protected ListMultimap mappings; + protected Set fileSet; + + private List endpointAffinities; + private ParquetGroupScanStatistics parquetGroupScanStatistics; + + protected AbstractParquetGroupScan(String userName, List columns, List entries, LogicalExpression filter) { + super(userName); + this.columns = columns; + this.entries = entries; + this.filter = filter; + } + + // immutable copy constructor + protected AbstractParquetGroupScan(AbstractParquetGroupScan that) { + super(that); + this.columns = that.columns == null ? null : new ArrayList<>(that.columns); + this.parquetTableMetadata = that.parquetTableMetadata; + this.rowGroupInfos = that.rowGroupInfos == null ? null : new ArrayList<>(that.rowGroupInfos); + this.filter = that.filter; + this.endpointAffinities = that.endpointAffinities == null ? null : new ArrayList<>(that.endpointAffinities); + this.mappings = that.mappings == null ? null : ArrayListMultimap.create(that.mappings); + this.parquetGroupScanStatistics = that.parquetGroupScanStatistics == null ? null : new ParquetGroupScanStatistics(that.parquetGroupScanStatistics); + this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet); + this.entries = that.entries == null ? null : new ArrayList<>(that.entries); + } + + @JsonProperty + public List getColumns() { + return columns; + } + + @JsonProperty + public List getEntries() { + return entries; + } + + @JsonIgnore + @Override + public Collection getFiles() { + return fileSet; + } + + @Override + public boolean hasFiles() { + return true; + } + + @Override + public boolean canPushdownProjects(List columns) { + return true; + } + + /** + * Return column value count for the specified column. + * If does not contain such column, return 0. + * Is used when applying convert to direct scan rule. + * + * @param column column schema path + * @return column value count + */ + @Override + public long getColumnValueCount(SchemaPath column) { + return parquetGroupScanStatistics.getColumnValueCount(column); + } + + /** + * Calculates the affinity each endpoint has for this scan, + * by adding up the affinity each endpoint has for each rowGroup. + * + * @return a list of EndpointAffinity objects + */ + @Override + public List getOperatorAffinity() { + return endpointAffinities; + } + + @Override + public void applyAssignments(List incomingEndpoints) { + this.mappings = AssignmentCreator.getMappings(incomingEndpoints, rowGroupInfos); + } + + @Override + public int getMaxParallelizationWidth() { + return rowGroupInfos.size(); + } + + @Override + public String getDigest() { + return toString(); + } + + @Override + public ScanStats getScanStats() { + int columnCount = columns == null ? 20 : columns.size(); + long rowCount = parquetGroupScanStatistics.getRowCount(); + ScanStats scanStats = new ScanStats(GroupScanProperty.EXACT_ROW_COUNT, rowCount, 1, rowCount * columnCount); + logger.trace("Drill parquet scan statistics: {}", scanStats); + return scanStats; + } + + protected List getReadEntries(int minorFragmentId) { + assert minorFragmentId < mappings.size() : String + .format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", + mappings.size(), minorFragmentId); + + List rowGroupsForMinor = mappings.get(minorFragmentId); + + Preconditions.checkArgument(!rowGroupsForMinor.isEmpty(), + String.format("MinorFragmentId %d has no read entries assigned", minorFragmentId)); + + List entries = new ArrayList<>(); + for (RowGroupInfo rgi : rowGroupsForMinor) { + RowGroupReadEntry entry = new RowGroupReadEntry(rgi.getPath(), rgi.getStart(), rgi.getLength(), rgi.getRowGroupIndex(), rgi.getNumRecordsToRead()); + entries.add(entry); + } + return entries; + } + + // filter push down methods block start + @JsonProperty + @Override + public LogicalExpression getFilter() { + return filter; + } + + public void setFilter(LogicalExpression filter) { + this.filter = filter; + } + + @Override + public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities, + FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) { + + if (rowGroupInfos.size() == 1 || + ! (parquetTableMetadata.isRowGroupPrunable()) || + rowGroupInfos.size() > optionManager.getOption(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD) + ) { + // Stop pruning for 3 cases: + // - 1 single parquet file, + // - metadata does not have proper format to support row group level filter pruning, + // - # of row groups is beyond PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD. + return null; + } + + final Set schemaPathsInExpr = filterExpr.accept(new ParquetRGFilterEvaluator.FieldReferenceFinder(), null); + + final List qualifiedRGs = new ArrayList<>(rowGroupInfos.size()); + Set qualifiedFilePath = new HashSet<>(); // HashSet keeps a fileName unique. + + ParquetFilterPredicate filterPredicate = null; + + for (RowGroupInfo rowGroup : rowGroupInfos) { + final ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, columns); + List partitionValues = getPartitionValues(rowGroup); + Map implicitColValues = columnExplorer.populateImplicitColumns(rowGroup.getPath(), partitionValues, supportsFileImplicitColumns()); + + ParquetMetaStatCollector statCollector = new ParquetMetaStatCollector( + parquetTableMetadata, + rowGroup.getColumns(), + implicitColValues); + + Map columnStatisticsMap = statCollector.collectColStat(schemaPathsInExpr); + + if (filterPredicate == null) { + ErrorCollector errorCollector = new ErrorCollectorImpl(); + LogicalExpression materializedFilter = ExpressionTreeMaterializer.materializeFilterExpr( + filterExpr, columnStatisticsMap, errorCollector, functionImplementationRegistry); + + if (errorCollector.hasErrors()) { + logger.error("{} error(s) encountered when materialize filter expression : {}", + errorCollector.getErrorCount(), errorCollector.toErrorString()); + return null; + } + logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter)); + + Set constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter); + filterPredicate = (ParquetFilterPredicate) ParquetFilterBuilder.buildParquetFilterPredicate( + materializedFilter, constantBoundaries, udfUtilities); + + if (filterPredicate == null) { + return null; + } + } + + if (ParquetRGFilterEvaluator.canDrop(filterPredicate, columnStatisticsMap, rowGroup.getRowCount())) { + continue; + } + + qualifiedRGs.add(rowGroup); + qualifiedFilePath.add(rowGroup.getPath()); // TODO : optimize when 1 file contains m row groups. + } + + if (qualifiedRGs.size() == rowGroupInfos.size() ) { + // There is no reduction of rowGroups. Return the original groupScan. + logger.debug("applyFilter does not have any pruning!"); + return null; + } else if (qualifiedFilePath.size() == 0) { + logger.debug("All rowgroups have been filtered out. Add back one to get schema from scannner"); + RowGroupInfo rg = rowGroupInfos.iterator().next(); + qualifiedFilePath.add(rg.getPath()); + qualifiedRGs.add(rg); + } + + logger.debug("applyFilter {} reduce parquet rowgroup # from {} to {}", + ExpressionStringBuilder.toString(filterExpr), rowGroupInfos.size(), qualifiedRGs.size()); + + try { + AbstractParquetGroupScan cloneGroupScan = cloneWithFileSelection(qualifiedFilePath); + cloneGroupScan.rowGroupInfos = qualifiedRGs; + cloneGroupScan.parquetGroupScanStatistics.collect(cloneGroupScan.rowGroupInfos, cloneGroupScan.parquetTableMetadata); + return cloneGroupScan; + + } catch (IOException e) { + logger.warn("Could not apply filter prune due to Exception : {}", e); + return null; + } + } + // filter push down methods block end + + // limit push down methods start + @Override + public boolean supportsLimitPushdown() { + return true; + } + + @Override + public GroupScan applyLimit(int maxRecords) { + maxRecords = Math.max(maxRecords, 1); // Make sure it request at least 1 row -> 1 rowGroup. + // further optimization : minimize # of files chosen, or the affinity of files chosen. + + // Calculate number of rowGroups to read based on maxRecords and update + // number of records to read for each of those rowGroups. + int index = updateRowGroupInfo(maxRecords); + + Set filePaths = rowGroupInfos.subList(0, index).stream() + .map(ReadEntryWithPath::getPath) + .collect(Collectors.toSet()); // HashSet keeps a filePath unique. + + // If there is no change in fileSet, no need to create new groupScan. + if (filePaths.size() == fileSet.size() ) { + // There is no reduction of rowGroups. Return the original groupScan. + logger.debug("applyLimit() does not apply!"); + return null; + } + + logger.debug("applyLimit() reduce parquet file # from {} to {}", fileSet.size(), filePaths.size()); + + try { + AbstractParquetGroupScan newScan = cloneWithFileSelection(filePaths); + newScan.updateRowGroupInfo(maxRecords); + return newScan; + } catch (IOException e) { + logger.warn("Could not apply rowcount based prune due to Exception : {}", e); + return null; + } + } + // limit push down methods end + + // partition pruning methods start + @Override + public List getPartitionColumns() { + return parquetGroupScanStatistics.getPartitionColumns(); + } + + @JsonIgnore + public TypeProtos.MajorType getTypeForColumn(SchemaPath schemaPath) { + return parquetGroupScanStatistics.getTypeForColumn(schemaPath); + } + + @JsonIgnore + public T getPartitionValue(String path, SchemaPath column, Class clazz) { + return clazz.cast(parquetGroupScanStatistics.getPartitionValue(path, column)); + } + + @JsonIgnore + public Set getFileSet() { + return fileSet; + } + // partition pruning methods end + + // helper method used for partition pruning and filter push down + @Override + public void modifyFileSelection(FileSelection selection) { + List files = selection.getFiles(); + fileSet = new HashSet<>(files); + entries = new ArrayList<>(files.size()); + + entries.addAll(files.stream() + .map(ReadEntryWithPath::new) + .collect(Collectors.toList())); + + rowGroupInfos = rowGroupInfos.stream() + .filter(rowGroupInfo -> fileSet.contains(rowGroupInfo.getPath())) + .collect(Collectors.toList()); + } + + + // protected methods block + protected void init() throws IOException { + initInternal(); + + assert parquetTableMetadata != null; + + if (fileSet == null) { + fileSet = new HashSet<>(); + fileSet.addAll(parquetTableMetadata.getFiles().stream() + .map((Function) ParquetFileMetadata::getPath) + .collect(Collectors.toSet())); + } + + Map hostEndpointMap = new HashMap<>(); + + for (CoordinationProtos.DrillbitEndpoint endpoint : getDrillbits()) { + hostEndpointMap.put(endpoint.getAddress(), endpoint); + } + + rowGroupInfos = new ArrayList<>(); + for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { + int rgIndex = 0; + for (RowGroupMetadata rg : file.getRowGroups()) { + RowGroupInfo rowGroupInfo = + new RowGroupInfo(file.getPath(), rg.getStart(), rg.getLength(), rgIndex, rg.getRowCount()); + EndpointByteMap endpointByteMap = new EndpointByteMapImpl(); + rg.getHostAffinity().keySet().stream() + .filter(hostEndpointMap::containsKey) + .forEach(host -> + endpointByteMap.add(hostEndpointMap.get(host), (long) (rg.getHostAffinity().get(host) * rg.getLength()))); + + rowGroupInfo.setEndpointByteMap(endpointByteMap); + rowGroupInfo.setColumns(rg.getColumns()); + rgIndex++; + rowGroupInfos.add(rowGroupInfo); + } + } + + this.endpointAffinities = AffinityCreator.getAffinityMap(rowGroupInfos); + this.parquetGroupScanStatistics = new ParquetGroupScanStatistics(rowGroupInfos, parquetTableMetadata); + } + + protected String getFilterString() { + return filter == null || filter.equals(ValueExpressions.BooleanExpression.TRUE) ? + "" : ExpressionStringBuilder.toString(this.filter); + } + + // abstract methods block start + protected abstract void initInternal() throws IOException; + protected abstract Collection getDrillbits(); + protected abstract AbstractParquetGroupScan cloneWithFileSelection(Collection filePaths) throws IOException; + protected abstract boolean supportsFileImplicitColumns(); + protected abstract List getPartitionValues(RowGroupInfo rowGroupInfo); + // abstract methods block end + + // private methods block start + /** + * Based on maxRecords to read for the scan, + * figure out how many rowGroups to read + * and update number of records to read for each of them. + * + * @param maxRecords max records to read + * @return total number of rowGroups to read + */ + private int updateRowGroupInfo(int maxRecords) { + long count = 0; + int index = 0; + for (RowGroupInfo rowGroupInfo : rowGroupInfos) { + long rowCount = rowGroupInfo.getRowCount(); + if (count + rowCount <= maxRecords) { + count += rowCount; + rowGroupInfo.setNumRecordsToRead(rowCount); + index++; + continue; + } else if (count < maxRecords) { + rowGroupInfo.setNumRecordsToRead(maxRecords - count); + index++; + } + break; + } + return index; + } + // private methods block end + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java new file mode 100644 index 00000000000..8726b9d1095 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.drill.common.expression.LogicalExpression; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.exec.physical.base.AbstractBase; +import org.apache.drill.exec.physical.base.GroupScan; +import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.physical.base.PhysicalVisitor; +import org.apache.drill.exec.physical.base.SubScan; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +public abstract class AbstractParquetRowGroupScan extends AbstractBase implements SubScan { + + protected final List rowGroupReadEntries; + protected final List columns; + protected final LogicalExpression filter; + + protected AbstractParquetRowGroupScan(String userName, + List rowGroupReadEntries, + List columns, + LogicalExpression filter) { + super(userName); + this.rowGroupReadEntries = rowGroupReadEntries; + this.columns = columns == null ? GroupScan.ALL_COLUMNS : columns; + this.filter = filter; + } + + @JsonProperty + public List getRowGroupReadEntries() { + return rowGroupReadEntries; + } + + @JsonProperty + public List getColumns() { + return columns; + } + + @JsonProperty + public LogicalExpression getFilter() { + return filter; + } + + @Override + public boolean isExecutable() { + return false; + } + + @Override + public T accept(PhysicalVisitor physicalVisitor, X value) throws E { + return physicalVisitor.visitSubScan(this, value); + } + + @Override + public Iterator iterator() { + return Collections.emptyIterator(); + } + + public abstract AbstractParquetRowGroupScan copy(List columns); + public abstract boolean areCorruptDatesAutoCorrected(); + @JsonIgnore + public abstract Configuration getFsConf(RowGroupReadEntry rowGroupReadEntry) throws IOException; + public abstract boolean supportsFileImplicitColumns(); + @JsonIgnore + public abstract List getPartitionValues(RowGroupReadEntry rowGroupReadEntry); + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java new file mode 100644 index 00000000000..6a320b85b38 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet; + +import com.google.common.base.Functions; +import com.google.common.base.Stopwatch; +import com.google.common.collect.Maps; +import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.ops.ExecutorFragmentContext; +import org.apache.drill.exec.ops.OperatorContext; +import org.apache.drill.exec.physical.impl.ScanBatch; +import org.apache.drill.exec.server.options.OptionManager; +import org.apache.drill.exec.store.ColumnExplorer; +import org.apache.drill.exec.store.RecordReader; +import org.apache.drill.exec.store.dfs.DrillFileSystem; +import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader; +import org.apache.drill.exec.store.parquet2.DrillParquetReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.CodecFactory; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public abstract class AbstractParquetScanBatchCreator { + + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractParquetScanBatchCreator.class); + + private static final String ENABLE_BYTES_READ_COUNTER = "parquet.benchmark.bytes.read"; + private static final String ENABLE_BYTES_TOTAL_COUNTER = "parquet.benchmark.bytes.total"; + private static final String ENABLE_TIME_READ_COUNTER = "parquet.benchmark.time.read"; + + protected ScanBatch getBatch(ExecutorFragmentContext context, AbstractParquetRowGroupScan rowGroupScan, OperatorContext oContext) throws ExecutionSetupException { + final ColumnExplorer columnExplorer = new ColumnExplorer(context.getOptions(), rowGroupScan.getColumns()); + + if (!columnExplorer.isStarQuery()) { + rowGroupScan = rowGroupScan.copy(columnExplorer.getTableColumns()); + rowGroupScan.setOperatorId(rowGroupScan.getOperatorId()); + } + + AbstractDrillFileSystemManager fsManager = getDrillFileSystemCreator(oContext, context.getOptions()); + + // keep footers in a map to avoid re-reading them + Map footers = new HashMap<>(); + List readers = new LinkedList<>(); + List> implicitColumns = new ArrayList<>(); + Map mapWithMaxColumns = new LinkedHashMap<>(); + for (RowGroupReadEntry rowGroup : rowGroupScan.getRowGroupReadEntries()) { + /* + Here we could store a map from file names to footers, to prevent re-reading the footer for each row group in a file + TODO - to prevent reading the footer again in the parquet record reader (it is read earlier in the ParquetStorageEngine) + we should add more information to the RowGroupInfo that will be populated upon the first read to + provide the reader with all of th file meta-data it needs + These fields will be added to the constructor below + */ + try { + Stopwatch timer = logger.isTraceEnabled() ? Stopwatch.createUnstarted() : null; + DrillFileSystem fs = fsManager.get(rowGroupScan.getFsConf(rowGroup), rowGroup.getPath()); + if (!footers.containsKey(rowGroup.getPath())) { + if (timer != null) { + timer.start(); + } + + ParquetMetadata footer = readFooter(fs.getConf(), rowGroup.getPath()); + if (timer != null) { + long timeToRead = timer.elapsed(TimeUnit.MICROSECONDS); + logger.trace("ParquetTrace,Read Footer,{},{},{},{},{},{},{}", "", rowGroup.getPath(), "", 0, 0, 0, timeToRead); + } + footers.put(rowGroup.getPath(), footer); + } + ParquetMetadata footer = footers.get(rowGroup.getPath()); + + boolean autoCorrectCorruptDates = rowGroupScan.areCorruptDatesAutoCorrected(); + ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = + ParquetReaderUtility.detectCorruptDates(footer, rowGroupScan.getColumns(), autoCorrectCorruptDates); + logger.debug("Contains corrupt dates: {}", containsCorruptDates); + + if (!context.getOptions().getBoolean(ExecConstants.PARQUET_NEW_RECORD_READER) && !isComplex(footer)) { + readers.add(new ParquetRecordReader(context, + rowGroup.getPath(), + rowGroup.getRowGroupIndex(), + rowGroup.getNumRecordsToRead(), + fs, + CodecFactory.createDirectCodecFactory(fs.getConf(), new ParquetDirectByteBufferAllocator(oContext.getAllocator()), 0), + footer, + rowGroupScan.getColumns(), + containsCorruptDates)); + } else { + readers.add(new DrillParquetReader(context, + footer, + rowGroup, + columnExplorer.getTableColumns(), + fs, + containsCorruptDates)); + } + + List partitionValues = rowGroupScan.getPartitionValues(rowGroup); + Map implicitValues = columnExplorer.populateImplicitColumns(rowGroup.getPath(), partitionValues, rowGroupScan.supportsFileImplicitColumns()); + implicitColumns.add(implicitValues); + if (implicitValues.size() > mapWithMaxColumns.size()) { + mapWithMaxColumns = implicitValues; + } + + } catch (IOException e) { + throw new ExecutionSetupException(e); + } + } + + // all readers should have the same number of implicit columns, add missing ones with value null + Map diff = Maps.transformValues(mapWithMaxColumns, Functions.constant((String) null)); + for (Map map : implicitColumns) { + map.putAll(Maps.difference(map, diff).entriesOnlyOnRight()); + } + + return new ScanBatch(context, oContext, readers, implicitColumns); + } + + protected abstract AbstractDrillFileSystemManager getDrillFileSystemCreator(OperatorContext operatorContext, OptionManager optionManager); + + private ParquetMetadata readFooter(Configuration conf, String path) throws IOException { + Configuration newConf = new Configuration(conf); + conf.setBoolean(ENABLE_BYTES_READ_COUNTER, false); + conf.setBoolean(ENABLE_BYTES_TOTAL_COUNTER, false); + conf.setBoolean(ENABLE_TIME_READ_COUNTER, false); + return ParquetFileReader.readFooter(newConf, new Path(path), ParquetMetadataConverter.NO_FILTER); + } + + private boolean isComplex(ParquetMetadata footer) { + MessageType schema = footer.getFileMetaData().getSchema(); + + for (Type type : schema.getFields()) { + if (!type.isPrimitive()) { + return true; + } + } + for (ColumnDescriptor col : schema.getColumns()) { + if (col.getMaxRepetitionLevel() > 0) { + return true; + } + } + return false; + } + + /** + * Helper class responsible for creating and managing DrillFileSystem. + */ + protected abstract class AbstractDrillFileSystemManager { + + protected final OperatorContext operatorContext; + + protected AbstractDrillFileSystemManager(OperatorContext operatorContext) { + this.operatorContext = operatorContext; + } + + protected abstract DrillFileSystem get(Configuration config, String path) throws ExecutionSetupException; + } + +} 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 deleted file mode 100644 index af98d332aa4..00000000000 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java +++ /dev/null @@ -1,2007 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.drill.exec.store.parquet; - -import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Iterator; - -import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; - -import org.apache.drill.common.expression.SchemaPath; -import org.apache.drill.common.util.DrillVersionInfo; -import org.apache.drill.exec.store.TimedRunnable; -import org.apache.drill.exec.util.DrillFileSystemUtil; -import org.apache.drill.exec.store.dfs.MetadataContext; -import org.apache.drill.exec.util.ImpersonationUtil; -import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.parquet.column.statistics.Statistics; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; -import org.apache.parquet.schema.Type; -import org.apache.commons.lang3.tuple.Pair; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonGenerator.Feature; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; -import com.fasterxml.jackson.databind.KeyDeserializer; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.fasterxml.jackson.module.afterburner.AfterburnerModule; -import com.google.common.base.Stopwatch; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -import javax.annotation.Nullable; - -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.SUPPORTED_VERSIONS; -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.V1; -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.V2; -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.V3; -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.V3_1; -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.V3_2; -import static org.apache.drill.exec.store.parquet.MetadataVersion.Constants.V3_3; - -public class Metadata { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class); - - 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; - private final ParquetFormatConfig formatConfig; - - private ParquetTableMetadataBase parquetTableMetadata; - private ParquetTableMetadataDirs parquetTableMetadataDirs; - - /** - * Create the parquet metadata file for the directory at the given path, and for any subdirectories - * - * @param fs - * @param path - * @throws IOException - */ - public static void createMeta(FileSystem fs, String path, ParquetFormatConfig formatConfig) throws IOException { - Metadata metadata = new Metadata(fs, formatConfig); - metadata.createMetaFilesRecursively(path); - } - - /** - * Get the parquet metadata for the parquet files in the given directory, including those in subdirectories - * - * @param fs - * @param path - * @return - * @throws IOException - */ - public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs, String path, ParquetFormatConfig formatConfig) - throws IOException { - Metadata metadata = new Metadata(fs, formatConfig); - return metadata.getParquetTableMetadata(path); - } - - /** - * Get the parquet metadata for a list of parquet files - * - * @param fs - * @param fileStatuses - * @return - * @throws IOException - */ - public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs, - List fileStatuses, ParquetFormatConfig formatConfig) throws IOException { - Metadata metadata = new Metadata(fs, formatConfig); - return metadata.getParquetTableMetadata(fileStatuses); - } - - /** - * Get the parquet metadata for the table by reading the metadata file - * - * @param fs current file system - * @param path The path to the metadata file, located in the directory that contains the parquet files - * @param metaContext metadata context - * @param formatConfig parquet format plugin configs - * @return parquet table metadata. Null if metadata cache is missing, unsupported or corrupted - */ - public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext, - ParquetFormatConfig formatConfig) { - if (ignoreReadingMetadata(metaContext, path)) { - return null; - } - Metadata metadata = new Metadata(fs, formatConfig); - metadata.readBlockMeta(path, false, metaContext); - return metadata.parquetTableMetadata; - } - - /** - * Get the parquet metadata for all subdirectories by reading the metadata file - * - * @param fs current file system - * @param path The path to the metadata file, located in the directory that contains the parquet files - * @param metaContext metadata context - * @param formatConfig parquet format plugin configs - * @return parquet metadata for a directory. Null if metadata cache is missing, unsupported or corrupted - */ - public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path, - MetadataContext metaContext, ParquetFormatConfig formatConfig) { - if (ignoreReadingMetadata(metaContext, path)) { - return null; - } - Metadata metadata = new Metadata(fs, formatConfig); - metadata.readBlockMeta(path, true, metaContext); - return metadata.parquetTableMetadataDirs; - } - - /** - * Ignore reading metadata files, if metadata is missing, unsupported or corrupted - * - * @param metaContext Metadata context - * @param path The path to the metadata file, located in the directory that contains the parquet files - * @return true if parquet metadata is missing or corrupted, false otherwise - */ - private static boolean ignoreReadingMetadata(MetadataContext metaContext, Path path) { - if (metaContext.isMetadataCacheCorrupted()) { - logger.warn("Ignoring of reading '{}' metadata file. Parquet metadata cache files are unsupported or corrupted. " + - "Query performance may be slow. Make sure the cache files are up-to-date by running the 'REFRESH TABLE " + - "METADATA' command", path); - return true; - } - return false; - } - - private Metadata(FileSystem fs, ParquetFormatConfig formatConfig) { - this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf()); - this.formatConfig = formatConfig; - } - - /** - * Create the parquet metadata files for the directory at the given path and for any subdirectories. - * Metadata cache files written to the disk contain relative paths. Returned Pair of metadata contains absolute paths. - * - * @param path to the directory of the parquet table - * @return Pair of parquet metadata. The left one is a parquet metadata for the table. The right one of the Pair is - * a metadata for all subdirectories (if they are present and there are no any parquet files in the - * {@code path} directory). - * @throws IOException if parquet metadata can't be serialized and written to the json file - */ - private Pair - createMetaFilesRecursively(final String path) throws IOException { - Stopwatch timer = Stopwatch.createStarted(); - List metaDataList = Lists.newArrayList(); - List directoryList = Lists.newArrayList(); - ConcurrentHashMap columnTypeInfoSet = - new ConcurrentHashMap<>(); - Path p = new Path(path); - FileStatus fileStatus = fs.getFileStatus(p); - assert fileStatus.isDirectory() : "Expected directory"; - - final List childFiles = Lists.newArrayList(); - - for (final FileStatus file : DrillFileSystemUtil.listAll(fs, p, false)) { - if (file.isDirectory()) { - ParquetTableMetadata_v3 subTableMetadata = (createMetaFilesRecursively(file.getPath().toString())).getLeft(); - metaDataList.addAll(subTableMetadata.files); - directoryList.addAll(subTableMetadata.directories); - directoryList.add(file.getPath().toString()); - // Merge the schema from the child level into the current level - //TODO: We need a merge method that merges two colums with the same name but different types - columnTypeInfoSet.putAll(subTableMetadata.columnTypeInfo); - } else { - childFiles.add(file); - } - } - ParquetTableMetadata_v3 parquetTableMetadata = new ParquetTableMetadata_v3(SUPPORTED_VERSIONS.last().toString(), - DrillVersionInfo.getVersion()); - if (childFiles.size() > 0) { - List childFilesMetadata = - getParquetFileMetadata_v3(parquetTableMetadata, childFiles); - metaDataList.addAll(childFilesMetadata); - // Note that we do not need to merge the columnInfo at this point. The columnInfo is already added - // to the parquetTableMetadata. - } - - parquetTableMetadata.directories = directoryList; - parquetTableMetadata.files = metaDataList; - // TODO: We need a merge method that merges two columns with the same name but different types - if (parquetTableMetadata.columnTypeInfo == null) { - parquetTableMetadata.columnTypeInfo = new ConcurrentHashMap<>(); - } - parquetTableMetadata.columnTypeInfo.putAll(columnTypeInfoSet); - - for (String oldName : OLD_METADATA_FILENAMES) { - fs.delete(new Path(p, oldName), false); - } - // relative paths in the metadata are only necessary for meta cache files. - ParquetTableMetadata_v3 metadataTableWithRelativePaths = - MetadataPathUtils.createMetadataWithRelativePaths(parquetTableMetadata, path); - writeFile(metadataTableWithRelativePaths, new Path(p, METADATA_FILENAME)); - - if (directoryList.size() > 0 && childFiles.size() == 0) { - ParquetTableMetadataDirs parquetTableMetadataDirsRelativePaths = - new ParquetTableMetadataDirs(metadataTableWithRelativePaths.directories); - writeFile(parquetTableMetadataDirsRelativePaths, new Path(p, METADATA_DIRECTORIES_FILENAME)); - logger.info("Creating metadata files recursively took {} ms", timer.elapsed(TimeUnit.MILLISECONDS)); - timer.stop(); - ParquetTableMetadataDirs parquetTableMetadataDirs = new ParquetTableMetadataDirs(directoryList); - return Pair.of(parquetTableMetadata, parquetTableMetadataDirs); - } - List emptyDirList = Lists.newArrayList(); - logger.info("Creating metadata files recursively took {} ms", timer.elapsed(TimeUnit.MILLISECONDS)); - timer.stop(); - return Pair.of(parquetTableMetadata, new ParquetTableMetadataDirs(emptyDirList)); - } - - /** - * Get the parquet metadata for the parquet files in a directory. - * - * @param path the path of the directory - * @return metadata object for an entire parquet directory structure - * @throws IOException in case of problems during accessing files - */ - private ParquetTableMetadata_v3 getParquetTableMetadata(String path) throws IOException { - Path p = new Path(path); - FileStatus fileStatus = fs.getFileStatus(p); - final Stopwatch watch = Stopwatch.createStarted(); - List fileStatuses = new ArrayList<>(); - if (fileStatus.isFile()) { - fileStatuses.add(fileStatus); - } else { - fileStatuses.addAll(DrillFileSystemUtil.listFiles(fs, p, true)); - } - logger.info("Took {} ms to get file statuses", watch.elapsed(TimeUnit.MILLISECONDS)); - watch.reset(); - watch.start(); - ParquetTableMetadata_v3 metadata_v3 = getParquetTableMetadata(fileStatuses); - logger.info("Took {} ms to read file metadata", watch.elapsed(TimeUnit.MILLISECONDS)); - return metadata_v3; - } - - /** - * Get the parquet metadata for a list of parquet files - * - * @param fileStatuses List of file statuses - * @return parquet table metadata object - * @throws IOException if parquet file metadata can't be obtained - */ - private ParquetTableMetadata_v3 getParquetTableMetadata(List fileStatuses) - throws IOException { - ParquetTableMetadata_v3 tableMetadata = new ParquetTableMetadata_v3(SUPPORTED_VERSIONS.last().toString(), - DrillVersionInfo.getVersion()); - List fileMetadataList = getParquetFileMetadata_v3(tableMetadata, fileStatuses); - tableMetadata.files = fileMetadataList; - tableMetadata.directories = new ArrayList(); - return tableMetadata; - } - - /** - * Get a list of file metadata for a list of parquet files - * - * @param parquetTableMetadata_v3 can store column schema info from all the files and row groups - * @param fileStatuses list of the parquet files statuses - * - * @return list of the parquet file metadata with absolute paths - * @throws IOException is thrown in case of issues while executing the list of runnables - */ - private List getParquetFileMetadata_v3( - ParquetTableMetadata_v3 parquetTableMetadata_v3, List fileStatuses) throws IOException { - List> gatherers = Lists.newArrayList(); - for (FileStatus file : fileStatuses) { - gatherers.add(new MetadataGatherer(parquetTableMetadata_v3, file)); - } - - List metaDataList = Lists.newArrayList(); - metaDataList.addAll(TimedRunnable.run("Fetch parquet metadata", logger, gatherers, 16)); - return metaDataList; - } - - /** - * TimedRunnable that reads the footer from parquet and collects file metadata - */ - private class MetadataGatherer extends TimedRunnable { - - private FileStatus fileStatus; - private ParquetTableMetadata_v3 parquetTableMetadata; - - public MetadataGatherer(ParquetTableMetadata_v3 parquetTableMetadata, FileStatus fileStatus) { - this.fileStatus = fileStatus; - this.parquetTableMetadata = parquetTableMetadata; - } - - @Override - protected ParquetFileMetadata_v3 runInner() throws Exception { - return getParquetFileMetadata_v3(parquetTableMetadata, fileStatus); - } - - @Override - protected IOException convertToIOException(Exception e) { - if (e instanceof IOException) { - return (IOException) e; - } else { - return new IOException(e); - } - } - } - - private OriginalType getOriginalType(Type type, String[] path, int depth) { - if (type.isPrimitive()) { - return type.getOriginalType(); - } - Type t = ((GroupType) type).getType(path[depth]); - return getOriginalType(t, path, depth + 1); - } - - private ColTypeInfo getColTypeInfo(MessageType schema, Type type, String[] path, int depth) { - if (type.isPrimitive()) { - PrimitiveType primitiveType = (PrimitiveType) type; - int precision = 0; - int scale = 0; - if (primitiveType.getDecimalMetadata() != null) { - precision = primitiveType.getDecimalMetadata().getPrecision(); - scale = primitiveType.getDecimalMetadata().getScale(); - } - - int repetitionLevel = schema.getMaxRepetitionLevel(path); - int definitionLevel = schema.getMaxDefinitionLevel(path); - - return new ColTypeInfo(type.getOriginalType(), precision, scale, repetitionLevel, definitionLevel); - } - Type t = ((GroupType) type).getType(path[depth]); - return getColTypeInfo(schema, t, path, depth + 1); - } - - private class ColTypeInfo { - public OriginalType originalType; - public int precision; - public int scale; - public int repetitionLevel; - public int definitionLevel; - - public ColTypeInfo(OriginalType originalType, int precision, int scale, int repetitionLevel, int definitionLevel) { - this.originalType = originalType; - this.precision = precision; - this.scale = scale; - this.repetitionLevel = repetitionLevel; - this.definitionLevel = definitionLevel; - } - } - - /** - * Get the metadata for a single file - */ - private ParquetFileMetadata_v3 getParquetFileMetadata_v3(ParquetTableMetadata_v3 parquetTableMetadata, - final FileStatus file) throws IOException, InterruptedException { - final ParquetMetadata metadata; - final UserGroupInformation processUserUgi = ImpersonationUtil.getProcessUserUGI(); - try { - metadata = processUserUgi.doAs(new PrivilegedExceptionAction() { - public ParquetMetadata run() throws Exception { - return ParquetFileReader.readFooter(fs.getConf(), file); - } - }); - } catch(Exception e) { - logger.error("Exception while reading footer of parquet file [Details - path: {}, owner: {}] as process user {}", - file.getPath(), file.getOwner(), processUserUgi.getShortUserName(), e); - throw e; - } - - MessageType schema = metadata.getFileMetaData().getSchema(); - -// Map originalTypeMap = Maps.newHashMap(); - Map colTypeInfoMap = Maps.newHashMap(); - schema.getPaths(); - for (String[] path : schema.getPaths()) { - colTypeInfoMap.put(SchemaPath.getCompoundPath(path), getColTypeInfo(schema, schema, path, 0)); - } - - List rowGroupMetadataList = Lists.newArrayList(); - - ArrayList ALL_COLS = new ArrayList<>(); - ALL_COLS.add(SchemaPath.STAR_COLUMN); - boolean autoCorrectCorruptDates = formatConfig.areCorruptDatesAutoCorrected(); - ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = ParquetReaderUtility.detectCorruptDates(metadata, ALL_COLS, autoCorrectCorruptDates); - if (logger.isDebugEnabled()) { - logger.debug(containsCorruptDates.toString()); - } - for (BlockMetaData rowGroup : metadata.getBlocks()) { - List columnMetadataList = Lists.newArrayList(); - long length = 0; - for (ColumnChunkMetaData col : rowGroup.getColumns()) { - ColumnMetadata_v3 columnMetadata; - - boolean statsAvailable = (col.getStatistics() != null && !col.getStatistics().isEmpty()); - - Statistics stats = col.getStatistics(); - String[] columnName = col.getPath().toArray(); - SchemaPath columnSchemaName = SchemaPath.getCompoundPath(columnName); - ColTypeInfo colTypeInfo = colTypeInfoMap.get(columnSchemaName); - - ColumnTypeMetadata_v3 columnTypeMetadata = - new ColumnTypeMetadata_v3(columnName, col.getType(), colTypeInfo.originalType, - colTypeInfo.precision, colTypeInfo.scale, colTypeInfo.repetitionLevel, colTypeInfo.definitionLevel); - - if (parquetTableMetadata.columnTypeInfo == null) { - parquetTableMetadata.columnTypeInfo = new ConcurrentHashMap<>(); - } - // Save the column schema info. We'll merge it into one list - parquetTableMetadata.columnTypeInfo - .put(new ColumnTypeMetadata_v3.Key(columnTypeMetadata.name), columnTypeMetadata); - if (statsAvailable) { - // Write stats when they are not null - Object minValue = null; - Object maxValue = null; - if (stats.genericGetMax() != null && stats.genericGetMin() != null ) { - minValue = stats.genericGetMin(); - maxValue = stats.genericGetMax(); - if (containsCorruptDates == ParquetReaderUtility.DateCorruptionStatus.META_SHOWS_CORRUPTION - && columnTypeMetadata.originalType == OriginalType.DATE) { - minValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) minValue); - maxValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) maxValue); - } - - } - columnMetadata = - new ColumnMetadata_v3(columnTypeMetadata.name, col.getType(), minValue, maxValue, stats.getNumNulls()); - } else { - columnMetadata = new ColumnMetadata_v3(columnTypeMetadata.name, col.getType(), null, null, null); - } - columnMetadataList.add(columnMetadata); - length += col.getTotalSize(); - } - - // DRILL-5009: Skip the RowGroup if it is empty - // Note we still read the schema even if there are no values in the RowGroup - if (rowGroup.getRowCount() == 0) { - continue; - } - RowGroupMetadata_v3 rowGroupMeta = - new RowGroupMetadata_v3(rowGroup.getStartingPos(), length, rowGroup.getRowCount(), - getHostAffinity(file, rowGroup.getStartingPos(), length), columnMetadataList); - - rowGroupMetadataList.add(rowGroupMeta); - } - String path = Path.getPathWithoutSchemeAndAuthority(file.getPath()).toString(); - - return new ParquetFileMetadata_v3(path, file.getLen(), rowGroupMetadataList); - } - - /** - * Get the host affinity for a row group - * - * @param fileStatus the parquet file - * @param start the start of the row group - * @param length the length of the row group - * @return - * @throws IOException - */ - private Map getHostAffinity(FileStatus fileStatus, long start, long length) - throws IOException { - BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, start, length); - Map hostAffinityMap = Maps.newHashMap(); - for (BlockLocation blockLocation : blockLocations) { - for (String host : blockLocation.getHosts()) { - Float currentAffinity = hostAffinityMap.get(host); - float blockStart = blockLocation.getOffset(); - float blockEnd = blockStart + blockLocation.getLength(); - float rowGroupEnd = start + length; - Float newAffinity = (blockLocation.getLength() - (blockStart < start ? start - blockStart : 0) - - (blockEnd > rowGroupEnd ? blockEnd - rowGroupEnd : 0)) / length; - if (currentAffinity != null) { - hostAffinityMap.put(host, currentAffinity + newAffinity); - } else { - hostAffinityMap.put(host, newAffinity); - } - } - } - return hostAffinityMap; - } - - /** - * Serialize parquet metadata to json and write to a file - * - * @param parquetTableMetadata - * @param p - * @throws IOException - */ - private void writeFile(ParquetTableMetadata_v3 parquetTableMetadata, 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(); - module.addSerializer(ColumnMetadata_v3.class, new ColumnMetadata_v3.Serializer()); - mapper.registerModule(module); - FSDataOutputStream os = fs.create(p); - mapper.writerWithDefaultPrettyPrinter().writeValue(os, parquetTableMetadata); - os.flush(); - 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 - * - * @param path to metadata file - * @param dirsOnly true for {@link Metadata#METADATA_DIRECTORIES_FILENAME} - * or false for {@link Metadata#METADATA_FILENAME} files reading - * @param metaContext current metadata context - * @throws IOException if metadata file can't be read or updated - */ - private void readBlockMeta(Path path, boolean dirsOnly, MetadataContext metaContext) { - Stopwatch timer = Stopwatch.createStarted(); - Path metadataParentDir = Path.getPathWithoutSchemeAndAuthority(path.getParent()); - String metadataParentDirPath = metadataParentDir.toUri().getPath(); - ObjectMapper mapper = new ObjectMapper(); - - final SimpleModule serialModule = new SimpleModule(); - serialModule.addDeserializer(SchemaPath.class, new SchemaPath.De()); - serialModule.addKeyDeserializer(ColumnTypeMetadata_v2.Key.class, new ColumnTypeMetadata_v2.Key.DeSerializer()); - serialModule.addKeyDeserializer(ColumnTypeMetadata_v3.Key.class, new ColumnTypeMetadata_v3.Key.DeSerializer()); - - AfterburnerModule module = new AfterburnerModule(); - module.setUseOptimizedBeanDeserializer(true); - - mapper.registerModule(serialModule); - mapper.registerModule(module); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - try (FSDataInputStream is = fs.open(path)) { - boolean alreadyCheckedModification = false; - boolean newMetadata = false; - alreadyCheckedModification = metaContext.getStatus(metadataParentDirPath); - - if (dirsOnly) { - parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class); - logger.info("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS)); - timer.stop(); - parquetTableMetadataDirs.updateRelativePaths(metadataParentDirPath); - if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), path, metadataParentDir, metaContext)) { - parquetTableMetadataDirs = - (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString())).getRight(); - newMetadata = true; - } - } else { - parquetTableMetadata = mapper.readValue(is, ParquetTableMetadataBase.class); - logger.info("Took {} ms to read metadata from cache file", timer.elapsed(TimeUnit.MILLISECONDS)); - timer.stop(); - if (new MetadataVersion(parquetTableMetadata.getMetadataVersion()).compareTo(new MetadataVersion(3, 0)) >= 0) { - ((ParquetTableMetadata_v3) parquetTableMetadata).updateRelativePaths(metadataParentDirPath); - } - if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), path, metadataParentDir, metaContext)) { - parquetTableMetadata = - (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString())).getLeft(); - newMetadata = true; - } - - // DRILL-5009: Remove the RowGroup if it is empty - List files = parquetTableMetadata.getFiles(); - for (ParquetFileMetadata file : files) { - List rowGroups = file.getRowGroups(); - for (Iterator iter = rowGroups.iterator(); iter.hasNext(); ) { - RowGroupMetadata r = iter.next(); - if (r.getRowCount() == 0) { - iter.remove(); - } - } - } - - } - if (newMetadata) { - // if new metadata files were created, invalidate the existing metadata context - metaContext.clear(); - } - } catch (IOException e) { - logger.error("Failed to read '{}' metadata file", path, e); - metaContext.setMetadataCacheCorrupted(true); - } - } - - /** - * 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 - * - * @param directories List of directories - * @param metaFilePath path of parquet metadata cache file - * @return true if metadata needs to be updated, false otherwise - * @throws IOException if some resources are not accessible - */ - private boolean tableModified(List directories, Path metaFilePath, Path parentDir, MetadataContext metaContext) - throws IOException { - - Stopwatch timer = Stopwatch.createStarted(); - - metaContext.setStatus(parentDir.toUri().getPath()); - long metaFileModifyTime = fs.getFileStatus(metaFilePath).getModificationTime(); - FileStatus directoryStatus = fs.getFileStatus(parentDir); - int numDirs = 1; - if (directoryStatus.getModificationTime() > metaFileModifyTime) { - logger.info("Directory {} was modified. Took {} ms to check modification time of {} directories", directoryStatus.getPath().toString(), - timer.elapsed(TimeUnit.MILLISECONDS), - numDirs); - timer.stop(); - return true; - } - for (String directory : directories) { - numDirs++; - metaContext.setStatus(directory); - directoryStatus = fs.getFileStatus(new Path(directory)); - if (directoryStatus.getModificationTime() > metaFileModifyTime) { - logger.info("Directory {} was modified. Took {} ms to check modification time of {} directories", directoryStatus.getPath().toString(), - timer.elapsed(TimeUnit.MILLISECONDS), - numDirs); - timer.stop(); - return true; - } - } - logger.info("No directories were modified. Took {} ms to check modification time of {} directories", timer.elapsed(TimeUnit.MILLISECONDS), numDirs); - timer.stop(); - return false; - } - - /** - * Basic class for parquet metadata. Inheritors of this class are json serializable structures which contain - * different metadata versions for an entire parquet directory structure - *

- * If any new code changes affect on the metadata files content, please update metadata version in such manner: - * Bump up metadata major version if metadata structure is changed. - * Bump up metadata minor version if only metadata content is changed, but metadata structure is the same. - *

- * Note: keep metadata versions synchronized with {@link MetadataVersion.Constants} - */ - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, - include = JsonTypeInfo.As.PROPERTY, - property = "metadata_version", - visible = true) - @JsonSubTypes({ - @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name = V1), - @JsonSubTypes.Type(value = ParquetTableMetadata_v2.class, name = V2), - @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name = V3), - @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name = V3_1), - @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name = V3_2), - @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name = V3_3) - }) - public static abstract class ParquetTableMetadataBase { - - @JsonIgnore public abstract List getDirectories(); - - @JsonIgnore public abstract List getFiles(); - - @JsonIgnore public abstract void assignFiles(List newFiles); - - public abstract boolean hasColumnMetadata(); - - @JsonIgnore public abstract PrimitiveTypeName getPrimitiveType(String[] columnName); - - @JsonIgnore public abstract OriginalType getOriginalType(String[] columnName); - - @JsonIgnore public abstract Integer getRepetitionLevel(String[] columnName); - - @JsonIgnore public abstract Integer getDefinitionLevel(String[] columnName); - - @JsonIgnore public abstract boolean isRowGroupPrunable(); - - @JsonIgnore public abstract ParquetTableMetadataBase clone(); - - @JsonIgnore public abstract String getDrillVersion(); - - @JsonIgnore public abstract String getMetadataVersion(); - } - - public static abstract class ParquetFileMetadata { - @JsonIgnore public abstract String getPath(); - - @JsonIgnore public abstract Long getLength(); - - @JsonIgnore public abstract List getRowGroups(); - } - - - public static abstract class RowGroupMetadata { - @JsonIgnore public abstract Long getStart(); - - @JsonIgnore public abstract Long getLength(); - - @JsonIgnore public abstract Long getRowCount(); - - @JsonIgnore public abstract Map getHostAffinity(); - - @JsonIgnore public abstract List getColumns(); - } - - - public static abstract class ColumnMetadata { - public abstract String[] getName(); - - public abstract Long getNulls(); - - public abstract boolean hasSingleValue(long rowCount); - - public abstract Object getMinValue(); - - public abstract Object getMaxValue(); - - /** - * Set the max value recorded in the parquet metadata statistics. - * - * This object would just be immutable, but due to Drill-4203 we need to correct - * date values that had been corrupted by earlier versions of Drill. - */ - public abstract void setMax(Object newMax); - - /** - * Set the min value recorded in the parquet metadata statistics. - * - * This object would just be immutable, but due to Drill-4203 we need to correct - * date values that had been corrupted by earlier versions of Drill. - */ - public abstract void setMin(Object newMax); - - public abstract PrimitiveTypeName getPrimitiveType(); - - 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; - } - - /** If directories list contains relative paths, update it to absolute ones - * @param baseDir base parent directory - */ - @JsonIgnore public void updateRelativePaths(String baseDir) { - this.directories = MetadataPathUtils.convertToAbsolutePaths(directories, baseDir); - } - } - - @JsonTypeName(V1) - public static class ParquetTableMetadata_v1 extends ParquetTableMetadataBase { - @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion; - @JsonProperty List files; - @JsonProperty List directories; - - public ParquetTableMetadata_v1() { - super(); - } - - public ParquetTableMetadata_v1(String metadataVersion, List files, List directories) { - this.metadataVersion = metadataVersion; - this.files = files; - this.directories = directories; - } - - @JsonIgnore @Override public List getDirectories() { - return directories; - } - - @JsonIgnore @Override public List getFiles() { - return files; - } - - @JsonIgnore @Override public void assignFiles(List newFiles) { - this.files = (List) newFiles; - } - - @Override public boolean hasColumnMetadata() { - return false; - } - - @JsonIgnore @Override public PrimitiveTypeName getPrimitiveType(String[] columnName) { - return null; - } - - @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) { - return null; - } - - @JsonIgnore @Override - public Integer getRepetitionLevel(String[] columnName) { - return null; - } - - @JsonIgnore @Override - public Integer getDefinitionLevel(String[] columnName) { - return null; - } - - @JsonIgnore @Override - public boolean isRowGroupPrunable() { - return false; - } - - @JsonIgnore @Override public ParquetTableMetadataBase clone() { - return new ParquetTableMetadata_v1(metadataVersion, files, directories); - } - - @JsonIgnore @Override - public String getDrillVersion() { - return null; - } - - @JsonIgnore @Override public String getMetadataVersion() { - return metadataVersion; - } - } - - - /** - * Struct which contains the metadata for a single parquet file - */ - public static class ParquetFileMetadata_v1 extends ParquetFileMetadata { - @JsonProperty - public String path; - @JsonProperty - public Long length; - @JsonProperty - public List rowGroups; - - public ParquetFileMetadata_v1() { - super(); - } - - public ParquetFileMetadata_v1(String path, Long length, List rowGroups) { - this.path = path; - this.length = length; - this.rowGroups = rowGroups; - } - - @Override - public String toString() { - return String.format("path: %s rowGroups: %s", path, rowGroups); - } - - @JsonIgnore @Override public String getPath() { - return path; - } - - @JsonIgnore @Override public Long getLength() { - return length; - } - - @JsonIgnore @Override public List getRowGroups() { - return rowGroups; - } - } - - - /** - * A struct that contains the metadata for a parquet row group - */ - public static class RowGroupMetadata_v1 extends RowGroupMetadata { - @JsonProperty - public Long start; - @JsonProperty - public Long length; - @JsonProperty - public Long rowCount; - @JsonProperty - public Map hostAffinity; - @JsonProperty - public List columns; - - public RowGroupMetadata_v1() { - super(); - } - - public RowGroupMetadata_v1(Long start, Long length, Long rowCount, Map hostAffinity, - List columns) { - this.start = start; - this.length = length; - this.rowCount = rowCount; - this.hostAffinity = hostAffinity; - this.columns = columns; - } - - @Override public Long getStart() { - return start; - } - - @Override public Long getLength() { - return length; - } - - @Override public Long getRowCount() { - return rowCount; - } - - @Override public Map getHostAffinity() { - return hostAffinity; - } - - @Override public List getColumns() { - return columns; - } - } - - - /** - * A struct that contains the metadata for a column in a parquet file - */ - public static class ColumnMetadata_v1 extends ColumnMetadata { - @JsonProperty - public SchemaPath name; - @JsonProperty - public PrimitiveTypeName primitiveType; - @JsonProperty - public OriginalType originalType; - @JsonProperty - public Long nulls; - - // JsonProperty for these are associated with the getters and setters - public Object max; - public Object min; - - - public ColumnMetadata_v1() { - super(); - } - - public ColumnMetadata_v1(SchemaPath name, PrimitiveTypeName primitiveType, OriginalType originalType, - Object max, Object min, Long nulls) { - this.name = name; - this.primitiveType = primitiveType; - this.originalType = originalType; - this.max = max; - this.min = min; - this.nulls = nulls; - } - - @JsonProperty(value = "min") - public Object getMin() { - if (primitiveType == PrimitiveTypeName.BINARY && min != null) { - return new String(((Binary) min).getBytes()); - } - return min; - } - - @JsonProperty(value = "max") - public Object getMax() { - if (primitiveType == PrimitiveTypeName.BINARY && max != null) { - return new String(((Binary) max).getBytes()); - } - return max; - } - - @Override public PrimitiveTypeName getPrimitiveType() { - return primitiveType; - } - - @Override public OriginalType getOriginalType() { - return originalType; - } - - /** - * setter used during deserialization of the 'min' field of the metadata cache file. - * - * @param min - */ - @JsonProperty(value = "min") - public void setMin(Object min) { - this.min = min; - } - - /** - * setter used during deserialization of the 'max' field of the metadata cache file. - * - * @param max - */ - @JsonProperty(value = "max") - public void setMax(Object max) { - this.max = max; - } - - @Override public String[] getName() { - String[] s = new String[1]; - String nameString = name.toString(); - // Strip out the surrounding backticks. - s[0]=nameString.substring(1, nameString.length()-1); - return s; - } - - @Override public Long getNulls() { - return nulls; - } - - /** - * Checks that the column chunk has a single value. - * Returns {@code true} if {@code min} and {@code max} are the same but not null - * and nulls count is 0 or equal to the rows count. - *

- * Returns {@code true} if {@code min} and {@code max} are null and the number of null values - * in the column chunk is equal to the rows count. - *

- * Comparison of nulls and rows count is needed for the cases: - *

    - *
  • column with primitive type has single value and null values
  • - * - *
  • column with primitive type has only null values, min/max couldn't be null, - * but column has single value
  • - *
- * - * @param rowCount rows count in column chunk - * @return true if column has single value - */ - @Override - public boolean hasSingleValue(long rowCount) { - if (nulls != null) { - if (min != null) { - // Objects.deepEquals() is used here, since min and max may be byte arrays - return Objects.deepEquals(min, max) && (nulls == 0 || nulls == rowCount); - } else { - return nulls == rowCount && max == null; - } - } - return false; - } - - @Override public Object getMinValue() { - return min; - } - - @Override public Object getMaxValue() { - return max; - } - - } - - /** - * Struct which contains the metadata for an entire parquet directory structure - */ - @JsonTypeName(V2) public static class ParquetTableMetadata_v2 extends ParquetTableMetadataBase { - @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion; - /* - ColumnTypeInfo is schema information from all the files and row groups, merged into - one. To get this info, we pass the ParquetTableMetadata object all the way dow to the - RowGroup and the column type is built there as it is read from the footer. - */ - @JsonProperty public ConcurrentHashMap columnTypeInfo; - @JsonProperty List files; - @JsonProperty List directories; - @JsonProperty String drillVersion; - - public ParquetTableMetadata_v2() { - super(); - } - - public ParquetTableMetadata_v2(String metadataVersion, String drillVersion) { - this.metadataVersion = metadataVersion; - this.drillVersion = drillVersion; - } - - public ParquetTableMetadata_v2(String metadataVersion, ParquetTableMetadataBase parquetTable, - List files, List directories, String drillVersion) { - this.metadataVersion = metadataVersion; - this.files = files; - this.directories = directories; - this.columnTypeInfo = ((ParquetTableMetadata_v2) parquetTable).columnTypeInfo; - this.drillVersion = drillVersion; - } - - public ParquetTableMetadata_v2(String metadataVersion, List files, List directories, - ConcurrentHashMap columnTypeInfo, String drillVersion) { - this.metadataVersion = metadataVersion; - this.files = files; - this.directories = directories; - this.columnTypeInfo = columnTypeInfo; - this.drillVersion = drillVersion; - } - - public ColumnTypeMetadata_v2 getColumnTypeInfo(String[] name) { - return columnTypeInfo.get(new ColumnTypeMetadata_v2.Key(name)); - } - - @JsonIgnore @Override public List getDirectories() { - return directories; - } - - @JsonIgnore @Override public List getFiles() { - return files; - } - - @JsonIgnore @Override public void assignFiles(List newFiles) { - this.files = (List) newFiles; - } - - @Override public boolean hasColumnMetadata() { - return true; - } - - @JsonIgnore @Override public PrimitiveTypeName getPrimitiveType(String[] columnName) { - return getColumnTypeInfo(columnName).primitiveType; - } - - @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) { - return getColumnTypeInfo(columnName).originalType; - } - - @JsonIgnore @Override - public Integer getRepetitionLevel(String[] columnName) { - return null; - } - - @JsonIgnore @Override - public Integer getDefinitionLevel(String[] columnName) { - return null; - } - - @JsonIgnore @Override - public boolean isRowGroupPrunable() { - return false; - } - - @JsonIgnore @Override public ParquetTableMetadataBase clone() { - return new ParquetTableMetadata_v2(metadataVersion, files, directories, columnTypeInfo, drillVersion); - } - - @JsonIgnore @Override - public String getDrillVersion() { - return drillVersion; - } - - @JsonIgnore @Override public String getMetadataVersion() { - return metadataVersion; - } - - } - - - /** - * Struct which contains the metadata for a single parquet file - */ - public static class ParquetFileMetadata_v2 extends ParquetFileMetadata { - @JsonProperty public String path; - @JsonProperty public Long length; - @JsonProperty public List rowGroups; - - public ParquetFileMetadata_v2() { - super(); - } - - public ParquetFileMetadata_v2(String path, Long length, List rowGroups) { - this.path = path; - this.length = length; - this.rowGroups = rowGroups; - } - - @Override public String toString() { - return String.format("path: %s rowGroups: %s", path, rowGroups); - } - - @JsonIgnore @Override public String getPath() { - return path; - } - - @JsonIgnore @Override public Long getLength() { - return length; - } - - @JsonIgnore @Override public List getRowGroups() { - return rowGroups; - } - } - - - /** - * A struct that contains the metadata for a parquet row group - */ - public static class RowGroupMetadata_v2 extends RowGroupMetadata { - @JsonProperty public Long start; - @JsonProperty public Long length; - @JsonProperty public Long rowCount; - @JsonProperty public Map hostAffinity; - @JsonProperty public List columns; - - public RowGroupMetadata_v2() { - super(); - } - - public RowGroupMetadata_v2(Long start, Long length, Long rowCount, Map hostAffinity, - List columns) { - this.start = start; - this.length = length; - this.rowCount = rowCount; - this.hostAffinity = hostAffinity; - this.columns = columns; - } - - @Override public Long getStart() { - return start; - } - - @Override public Long getLength() { - return length; - } - - @Override public Long getRowCount() { - return rowCount; - } - - @Override public Map getHostAffinity() { - return hostAffinity; - } - - @Override public List getColumns() { - return columns; - } - } - - - public static class ColumnTypeMetadata_v2 { - @JsonProperty public String[] name; - @JsonProperty public PrimitiveTypeName primitiveType; - @JsonProperty public OriginalType originalType; - - // Key to find by name only - @JsonIgnore private Key key; - - public ColumnTypeMetadata_v2() { - super(); - } - - public ColumnTypeMetadata_v2(String[] name, PrimitiveTypeName primitiveType, OriginalType originalType) { - this.name = name; - this.primitiveType = primitiveType; - this.originalType = originalType; - this.key = new Key(name); - } - - @JsonIgnore private Key key() { - return this.key; - } - - private static class Key { - private String[] name; - private int hashCode = 0; - - public Key(String[] name) { - this.name = name; - } - - @Override public int hashCode() { - if (hashCode == 0) { - hashCode = Arrays.hashCode(name); - } - return hashCode; - } - - @Override public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - final Key other = (Key) obj; - return Arrays.equals(this.name, other.name); - } - - @Override public String toString() { - String s = null; - for (String namePart : name) { - if (s != null) { - s += "."; - s += namePart; - } else { - s = namePart; - } - } - return s; - } - - public static class DeSerializer extends KeyDeserializer { - - public DeSerializer() { - super(); - } - - @Override - public Object deserializeKey(String key, com.fasterxml.jackson.databind.DeserializationContext ctxt) - throws IOException, com.fasterxml.jackson.core.JsonProcessingException { - return new Key(key.split("\\.")); - } - } - } - } - - - /** - * A struct that contains the metadata for a column in a parquet file - */ - public static class ColumnMetadata_v2 extends ColumnMetadata { - // Use a string array for name instead of Schema Path to make serialization easier - @JsonProperty public String[] name; - @JsonProperty public Long nulls; - - public Object mxValue; - - @JsonIgnore private PrimitiveTypeName primitiveType; - - public ColumnMetadata_v2() { - super(); - } - - public ColumnMetadata_v2(String[] name, PrimitiveTypeName primitiveType, Object mxValue, Long nulls) { - this.name = name; - this.mxValue = mxValue; - this.nulls = nulls; - this.primitiveType = primitiveType; - } - - @JsonProperty(value = "mxValue") public void setMax(Object mxValue) { - this.mxValue = mxValue; - } - - @Override public String[] getName() { - return name; - } - - @Override public Long getNulls() { - return nulls; - } - - /** - * Checks that the column chunk has a single value. - * Returns {@code true} if {@code mxValue} is not null - * and nulls count is 0 or if nulls count is equal to the rows count. - *

- * Comparison of nulls and rows count is needed for the cases: - *

    - *
  • column with primitive type has single value and null values
  • - * - *
  • column with binary type has only null values, so column has single value
  • - *
- * - * @param rowCount rows count in column chunk - * @return true if column has single value - */ - @Override - public boolean hasSingleValue(long rowCount) { - return (mxValue != null && nulls == 0) || nulls == rowCount; - } - - @Override public Object getMinValue() { - return mxValue; - } - - @Override public Object getMaxValue() { - return mxValue; - } - - @Override - public void setMin(Object newMin) { - // noop - min value not stored in this version of the metadata - } - - @Override public PrimitiveTypeName getPrimitiveType() { - return primitiveType; - } - - @Override public OriginalType getOriginalType() { - return null; - } - - public static class DeSerializer extends JsonDeserializer { - @Override public ColumnMetadata_v2 deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException, JsonProcessingException { - return null; - } - } - - - // We use a custom serializer and write only non null values. - public static class Serializer extends JsonSerializer { - @Override - public void serialize(ColumnMetadata_v2 value, JsonGenerator jgen, SerializerProvider provider) - throws IOException, JsonProcessingException { - jgen.writeStartObject(); - jgen.writeArrayFieldStart("name"); - for (String n : value.name) { - jgen.writeString(n); - } - jgen.writeEndArray(); - if (value.mxValue != null) { - Object val; - if (value.primitiveType == PrimitiveTypeName.BINARY && value.mxValue != null) { - val = new String(((Binary) value.mxValue).getBytes()); - } else { - val = value.mxValue; - } - jgen.writeObjectField("mxValue", val); - } - if (value.nulls != null) { - jgen.writeObjectField("nulls", value.nulls); - } - jgen.writeEndObject(); - } - } - - } - - @JsonTypeName(V3_3) - public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase { - @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion; - /* - ColumnTypeInfo is schema information from all the files and row groups, merged into - one. To get this info, we pass the ParquetTableMetadata object all the way dow to the - RowGroup and the column type is built there as it is read from the footer. - */ - @JsonProperty public ConcurrentHashMap columnTypeInfo; - @JsonProperty List files; - @JsonProperty List directories; - @JsonProperty String drillVersion; - - /** - * Default constructor needed for deserialization from Parquet Metadata Cache Files - */ - public ParquetTableMetadata_v3() { - super(); - } - - /** - * Used for creating the Parquet Metadata cache file - * @param metadataVersion metadata version - * @param drillVersion apache drill version - */ - public ParquetTableMetadata_v3(String metadataVersion, String drillVersion) { - this.metadataVersion = metadataVersion; - this.drillVersion = drillVersion; - } - - public ParquetTableMetadata_v3(String metadataVersion, ParquetTableMetadataBase parquetTable, - List files, List directories, String drillVersion) { - this.metadataVersion = metadataVersion; - this.files = files; - this.directories = directories; - this.columnTypeInfo = ((ParquetTableMetadata_v3) parquetTable).columnTypeInfo; - this.drillVersion = drillVersion; - } - - public ParquetTableMetadata_v3(String metadataVersion, List files, List directories, - ConcurrentHashMap columnTypeInfo, - String drillVersion) { - this.metadataVersion = metadataVersion; - this.files = files; - this.directories = directories; - this.columnTypeInfo = columnTypeInfo; - this.drillVersion = drillVersion; - } - - public ColumnTypeMetadata_v3 getColumnTypeInfo(String[] name) { - return columnTypeInfo.get(new ColumnTypeMetadata_v3.Key(name)); - } - - @JsonIgnore @Override public List getDirectories() { - return directories; - } - - @JsonIgnore @Override public String getMetadataVersion() { - return metadataVersion; - } - - /** - * If directories list and file metadata list contain relative paths, update it to absolute ones - * @param baseDir base parent directory - */ - @JsonIgnore public void updateRelativePaths(String baseDir) { - // update directories paths to absolute ones - this.directories = MetadataPathUtils.convertToAbsolutePaths(directories, baseDir); - - // update files paths to absolute ones - this.files = MetadataPathUtils.convertToFilesWithAbsolutePaths(files, baseDir); - } - - @JsonIgnore @Override public List getFiles() { - return files; - } - - @JsonIgnore @Override public void assignFiles(List newFiles) { - this.files = (List) newFiles; - } - - @Override public boolean hasColumnMetadata() { - return true; - } - - @JsonIgnore @Override public PrimitiveTypeName getPrimitiveType(String[] columnName) { - return getColumnTypeInfo(columnName).primitiveType; - } - - @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) { - return getColumnTypeInfo(columnName).originalType; - } - - @JsonIgnore @Override - public Integer getRepetitionLevel(String[] columnName) { - return getColumnTypeInfo(columnName).repetitionLevel; - } - - @JsonIgnore @Override - public Integer getDefinitionLevel(String[] columnName) { - return getColumnTypeInfo(columnName).definitionLevel; - } - - @JsonIgnore @Override - public boolean isRowGroupPrunable() { - return true; - } - - @JsonIgnore @Override public ParquetTableMetadataBase clone() { - return new ParquetTableMetadata_v3(metadataVersion, files, directories, columnTypeInfo, drillVersion); - } - - @JsonIgnore @Override - public String getDrillVersion() { - return drillVersion; - } - - } - - - /** - * Struct which contains the metadata for a single parquet file - */ - public static class ParquetFileMetadata_v3 extends ParquetFileMetadata { - @JsonProperty public String path; - @JsonProperty public Long length; - @JsonProperty public List rowGroups; - - public ParquetFileMetadata_v3() { - super(); - } - - public ParquetFileMetadata_v3(String path, Long length, List rowGroups) { - this.path = path; - this.length = length; - this.rowGroups = rowGroups; - } - - @Override public String toString() { - return String.format("path: %s rowGroups: %s", path, rowGroups); - } - - @JsonIgnore @Override public String getPath() { - return path; - } - - @JsonIgnore @Override public Long getLength() { - return length; - } - - @JsonIgnore @Override public List getRowGroups() { - return rowGroups; - } - } - - - /** - * A struct that contains the metadata for a parquet row group - */ - public static class RowGroupMetadata_v3 extends RowGroupMetadata { - @JsonProperty public Long start; - @JsonProperty public Long length; - @JsonProperty public Long rowCount; - @JsonProperty public Map hostAffinity; - @JsonProperty public List columns; - - public RowGroupMetadata_v3() { - super(); - } - - public RowGroupMetadata_v3(Long start, Long length, Long rowCount, Map hostAffinity, - List columns) { - this.start = start; - this.length = length; - this.rowCount = rowCount; - this.hostAffinity = hostAffinity; - this.columns = columns; - } - - @Override public Long getStart() { - return start; - } - - @Override public Long getLength() { - return length; - } - - @Override public Long getRowCount() { - return rowCount; - } - - @Override public Map getHostAffinity() { - return hostAffinity; - } - - @Override public List getColumns() { - return columns; - } - } - - - public static class ColumnTypeMetadata_v3 { - @JsonProperty public String[] name; - @JsonProperty public PrimitiveTypeName primitiveType; - @JsonProperty public OriginalType originalType; - @JsonProperty public int precision; - @JsonProperty public int scale; - @JsonProperty public int repetitionLevel; - @JsonProperty public int definitionLevel; - - // Key to find by name only - @JsonIgnore private Key key; - - public ColumnTypeMetadata_v3() { - super(); - } - - public ColumnTypeMetadata_v3(String[] name, PrimitiveTypeName primitiveType, OriginalType originalType, int precision, int scale, int repetitionLevel, int definitionLevel) { - this.name = name; - this.primitiveType = primitiveType; - this.originalType = originalType; - this.precision = precision; - this.scale = scale; - this.repetitionLevel = repetitionLevel; - this.definitionLevel = definitionLevel; - this.key = new Key(name); - } - - @JsonIgnore private Key key() { - return this.key; - } - - private static class Key { - private SchemaPath name; - private int hashCode = 0; - - public Key(String[] name) { - this.name = SchemaPath.getCompoundPath(name); - } - - public Key(SchemaPath name) { - this.name = new SchemaPath(name); - } - - @Override public int hashCode() { - if (hashCode == 0) { - hashCode = name.hashCode(); - } - return hashCode; - } - - @Override public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - final Key other = (Key) obj; - return this.name.equals(other.name); - } - - @Override public String toString() { - return name.toString(); - } - - public static class DeSerializer extends KeyDeserializer { - - public DeSerializer() { - super(); - } - - @Override - public Object deserializeKey(String key, com.fasterxml.jackson.databind.DeserializationContext ctxt) - throws IOException, com.fasterxml.jackson.core.JsonProcessingException { - // key string should contain '`' char if the field was serialized as SchemaPath object - if (key.contains("`")) { - return new Key(SchemaPath.parseFromString(key)); - } - return new Key(key.split("\\.")); - } - } - } - } - - - /** - * A struct that contains the metadata for a column in a parquet file - */ - public static class ColumnMetadata_v3 extends ColumnMetadata { - // Use a string array for name instead of Schema Path to make serialization easier - @JsonProperty public String[] name; - @JsonProperty public Long nulls; - - public Object minValue; - public Object maxValue; - - @JsonIgnore private PrimitiveTypeName primitiveType; - - public ColumnMetadata_v3() { - super(); - } - - public ColumnMetadata_v3(String[] name, PrimitiveTypeName primitiveType, Object minValue, Object maxValue, Long nulls) { - this.name = name; - this.minValue = minValue; - this.maxValue = maxValue; - this.nulls = nulls; - this.primitiveType = primitiveType; - } - - @JsonProperty(value = "minValue") public void setMin(Object minValue) { - this.minValue = minValue; - } - - @JsonProperty(value = "maxValue") public void setMax(Object maxValue) { - this.maxValue = maxValue; - } - - @Override public String[] getName() { - return name; - } - - @Override public Long getNulls() { - return nulls; - } - - /** - * Checks that the column chunk has a single value. - * Returns {@code true} if {@code minValue} and {@code maxValue} are the same but not null - * and nulls count is 0 or equal to the rows count. - *

- * Returns {@code true} if {@code minValue} and {@code maxValue} are null and the number of null values - * in the column chunk is equal to the rows count. - *

- * Comparison of nulls and rows count is needed for the cases: - *

    - *
  • column with primitive type has single value and null values
  • - * - *
  • column with primitive type has only null values, min/max couldn't be null, - * but column has single value
  • - *
- * - * @param rowCount rows count in column chunk - * @return true if column has single value - */ - @Override - public boolean hasSingleValue(long rowCount) { - if (nulls != null) { - if (minValue != null) { - // Objects.deepEquals() is used here, since min and max may be byte arrays - return Objects.deepEquals(minValue, maxValue) && (nulls == 0 || nulls == rowCount); - } else { - return nulls == rowCount && maxValue == null; - } - } - return false; - } - - @Override public Object getMinValue() { - return minValue; - } - - @Override public Object getMaxValue() { - return maxValue; - } - - @Override public PrimitiveTypeName getPrimitiveType() { - return null; - } - - @Override public OriginalType getOriginalType() { - return null; - } - - public static class DeSerializer extends JsonDeserializer { - @Override public ColumnMetadata_v3 deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException, JsonProcessingException { - return null; - } - } - - - // We use a custom serializer and write only non null values. - public static class Serializer extends JsonSerializer { - @Override - public void serialize(ColumnMetadata_v3 value, JsonGenerator jgen, SerializerProvider provider) - throws IOException, JsonProcessingException { - jgen.writeStartObject(); - jgen.writeArrayFieldStart("name"); - for (String n : value.name) { - jgen.writeString(n); - } - jgen.writeEndArray(); - if (value.minValue != null) { - Object val; - if (value.primitiveType == PrimitiveTypeName.BINARY - || value.primitiveType == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { - val = ((Binary) value.minValue).getBytes(); - } else { - val = value.minValue; - } - jgen.writeObjectField("minValue", val); - } - if (value.maxValue != null) { - Object val; - if (value.primitiveType == PrimitiveTypeName.BINARY - || value.primitiveType == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { - val = ((Binary) value.maxValue).getBytes(); - } else { - val = value.maxValue; - } - jgen.writeObjectField("maxValue", val); - } - - if (value.nulls != null) { - jgen.writeObjectField("nulls", value.nulls); - } - jgen.writeEndObject(); - } - } - - } - - /** - * Util class that contains helper methods for converting paths in the table and directory metadata structures - */ - private static class MetadataPathUtils { - - /** - * Helper method that converts a list of relative paths to absolute ones - * - * @param paths list of relative paths - * @param baseDir base parent directory - * @return list of absolute paths - */ - private static List convertToAbsolutePaths(List paths, String baseDir) { - if (!paths.isEmpty()) { - List absolutePaths = Lists.newArrayList(); - for (String relativePath : paths) { - String absolutePath = (new Path(relativePath).isAbsolute()) ? relativePath - : new Path(baseDir, relativePath).toUri().getPath(); - absolutePaths.add(absolutePath); - } - return absolutePaths; - } - return paths; - } - - /** - * Convert a list of files with relative paths to files with absolute ones - * - * @param files list of files with relative paths - * @param baseDir base parent directory - * @return list of files with absolute paths - */ - private static List convertToFilesWithAbsolutePaths( - List files, String baseDir) { - if (!files.isEmpty()) { - List filesWithAbsolutePaths = Lists.newArrayList(); - for (ParquetFileMetadata_v3 file : files) { - Path relativePath = new Path(file.getPath()); - // create a new file if old one contains a relative path, otherwise use an old file - ParquetFileMetadata_v3 fileWithAbsolutePath = (relativePath.isAbsolute()) ? file - : new ParquetFileMetadata_v3(new Path(baseDir, relativePath).toUri().getPath(), file.length, file.rowGroups); - filesWithAbsolutePaths.add(fileWithAbsolutePath); - } - return filesWithAbsolutePaths; - } - return files; - } - - /** - * Creates a new parquet table metadata from the {@code tableMetadataWithAbsolutePaths} parquet table. - * A new parquet table will contain relative paths for the files and directories. - * - * @param tableMetadataWithAbsolutePaths parquet table metadata with absolute paths for the files and directories - * @param baseDir base parent directory - * @return parquet table metadata with relative paths for the files and directories - */ - private static ParquetTableMetadata_v3 createMetadataWithRelativePaths( - ParquetTableMetadata_v3 tableMetadataWithAbsolutePaths, String baseDir) { - List directoriesWithRelativePaths = Lists.newArrayList(); - for (String directory : tableMetadataWithAbsolutePaths.getDirectories()) { - directoriesWithRelativePaths.add(relativize(baseDir, directory)) ; - } - List filesWithRelativePaths = Lists.newArrayList(); - for (ParquetFileMetadata_v3 file : tableMetadataWithAbsolutePaths.files) { - filesWithRelativePaths.add(new ParquetFileMetadata_v3( - relativize(baseDir, file.getPath()), file.length, file.rowGroups)); - } - return new ParquetTableMetadata_v3(SUPPORTED_VERSIONS.last().toString(), tableMetadataWithAbsolutePaths, - filesWithRelativePaths, directoriesWithRelativePaths, DrillVersionInfo.getVersion()); - } - - /** - * Constructs relative path from child full path and base path. Or return child path if the last one is already relative - * - * @param childPath full absolute path - * @param baseDir base path (the part of the Path, which should be cut off from child path) - * @return relative path - */ - private static String relativize(String baseDir, String childPath) { - Path fullPathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(childPath)); - Path basePathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(baseDir)); - - // Since hadoop Path hasn't relativize() we use uri.relativize() to get relative path - Path relativeFilePath = new Path(basePathWithoutSchemeAndAuthority.toUri() - .relativize(fullPathWithoutSchemeAndAuthority.toUri())); - if (relativeFilePath.isAbsolute()) { - throw new IllegalStateException(String.format("Path %s is not a subpath of %s.", - basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath())); - } - return relativeFilePath.toUri().getPath(); - } - } - -} - 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 9b0794d5119..f66cf303bcf 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 @@ -47,6 +47,7 @@ import org.apache.drill.exec.store.dfs.FileSelection; import org.apache.drill.exec.store.dfs.FileSystemConfig; import org.apache.drill.exec.store.dfs.FileSystemPlugin; +import org.apache.drill.exec.store.parquet.metadata.Metadata; import org.apache.drill.exec.util.DrillFileSystemUtil; import org.apache.drill.exec.store.dfs.FormatMatcher; import org.apache.drill.exec.store.dfs.FormatPlugin; @@ -54,7 +55,7 @@ import org.apache.drill.exec.store.dfs.MagicString; import org.apache.drill.exec.store.dfs.MetadataContext; import org.apache.drill.exec.store.mock.MockStorageEngine; -import org.apache.drill.exec.store.parquet.Metadata.ParquetTableMetadataDirs; +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; 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 a53587ce9ae..564b3bdb645 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 @@ -20,196 +20,106 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.exceptions.UserException; -import org.apache.drill.common.expression.ErrorCollector; -import org.apache.drill.common.expression.ErrorCollectorImpl; -import org.apache.drill.common.expression.ExpressionStringBuilder; import org.apache.drill.common.expression.LogicalExpression; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.expression.ValueExpressions; import org.apache.drill.common.logical.FormatPluginConfig; import org.apache.drill.common.logical.StoragePluginConfig; -import org.apache.drill.common.types.TypeProtos; -import org.apache.drill.common.types.TypeProtos.MajorType; -import org.apache.drill.common.types.TypeProtos.MinorType; -import org.apache.drill.common.types.Types; -import org.apache.drill.exec.compile.sig.ConstantExpressionIdentifier; -import org.apache.drill.exec.expr.ExpressionTreeMaterializer; -import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; -import org.apache.drill.exec.expr.stat.ParquetFilterPredicate; -import org.apache.drill.exec.ops.UdfUtilities; -import org.apache.drill.exec.physical.EndpointAffinity; -import org.apache.drill.exec.physical.PhysicalOperatorSetupException; -import org.apache.drill.exec.physical.base.AbstractFileGroupScan; import org.apache.drill.exec.physical.base.GroupScan; import org.apache.drill.exec.physical.base.PhysicalOperator; -import org.apache.drill.exec.physical.base.ScanStats; -import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty; -import org.apache.drill.exec.planner.physical.PlannerSettings; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; -import org.apache.drill.exec.server.options.OptionManager; import org.apache.drill.exec.store.ColumnExplorer; import org.apache.drill.exec.store.StoragePluginRegistry; import org.apache.drill.exec.store.dfs.DrillFileSystem; import org.apache.drill.exec.store.dfs.FileSelection; +import org.apache.drill.exec.store.parquet.metadata.Metadata; import org.apache.drill.exec.util.DrillFileSystemUtil; import org.apache.drill.exec.store.dfs.MetadataContext; import org.apache.drill.exec.store.dfs.MetadataContext.PruneStatus; -import org.apache.drill.exec.store.dfs.ReadEntryFromHDFS; import org.apache.drill.exec.store.dfs.ReadEntryWithPath; -import org.apache.drill.exec.store.dfs.easy.FileWork; -import org.apache.drill.exec.store.parquet.Metadata.ColumnMetadata; -import org.apache.drill.exec.store.parquet.Metadata.ParquetFileMetadata; -import org.apache.drill.exec.store.parquet.Metadata.ParquetTableMetadataBase; -import org.apache.drill.exec.store.parquet.Metadata.RowGroupMetadata; -import org.apache.drill.exec.store.parquet.stat.ColumnStatistics; -import org.apache.drill.exec.store.parquet.stat.ParquetMetaStatCollector; -import org.apache.drill.exec.store.schedule.AffinityCreator; -import org.apache.drill.exec.store.schedule.AssignmentCreator; -import org.apache.drill.exec.store.schedule.CompleteWork; -import org.apache.drill.exec.store.schedule.EndpointByteMap; -import org.apache.drill.exec.store.schedule.EndpointByteMapImpl; -import org.apache.drill.exec.util.DecimalUtility; +import org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; import org.apache.drill.exec.util.ImpersonationUtil; -import org.apache.drill.exec.vector.NullableBitVector; -import org.apache.drill.exec.vector.NullableBigIntVector; -import org.apache.drill.exec.vector.NullableDateVector; -import org.apache.drill.exec.vector.NullableDecimal18Vector; -import org.apache.drill.exec.vector.NullableFloat4Vector; -import org.apache.drill.exec.vector.NullableFloat8Vector; -import org.apache.drill.exec.vector.NullableIntVector; -import org.apache.drill.exec.vector.NullableIntervalVector; -import org.apache.drill.exec.vector.NullableSmallIntVector; -import org.apache.drill.exec.vector.NullableTimeStampVector; -import org.apache.drill.exec.vector.NullableTimeVector; -import org.apache.drill.exec.vector.NullableTinyIntVector; -import org.apache.drill.exec.vector.NullableUInt1Vector; -import org.apache.drill.exec.vector.NullableUInt2Vector; -import org.apache.drill.exec.vector.NullableUInt4Vector; -import org.apache.drill.exec.vector.NullableVarBinaryVector; -import org.apache.drill.exec.vector.NullableVarCharVector; -import org.apache.drill.exec.vector.ValueVector; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.joda.time.DateTimeConstants; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ListMultimap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Sets; @JsonTypeName("parquet-scan") -public class ParquetGroupScan extends AbstractFileGroupScan { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetGroupScan.class); +public class ParquetGroupScan extends AbstractParquetGroupScan { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetGroupScan.class); - private final List entries; private final ParquetFormatPlugin formatPlugin; private final ParquetFormatConfig formatConfig; private final DrillFileSystem fs; private final MetadataContext metaContext; + private boolean usedMetadataCache; // false by default + // may change when filter push down / partition pruning is applied private String selectionRoot; - - private boolean usedMetadataCache = false; - private List endpointAffinities; - private List columns; - private ListMultimap mappings; - private List rowGroupInfos; - private LogicalExpression filter; - - /** - * 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) - */ - private long rowCount; - - /* - * total number of non-null value for each column in parquet files. - */ - private Map columnValueCounts; - - @JsonCreator public ParquetGroupScan( // - @JsonProperty("userName") String userName, - @JsonProperty("entries") List entries,// - @JsonProperty("storage") StoragePluginConfig storageConfig, // - @JsonProperty("format") FormatPluginConfig formatConfig, // - @JacksonInject StoragePluginRegistry engineRegistry, // - @JsonProperty("columns") List columns, // - @JsonProperty("selectionRoot") String selectionRoot, // - @JsonProperty("cacheFileRoot") String cacheFileRoot, // - @JsonProperty("filter") LogicalExpression filter - ) throws IOException, ExecutionSetupException { - super(ImpersonationUtil.resolveUserName(userName)); - this.columns = columns; + private String cacheFileRoot; + + @JsonCreator + public ParquetGroupScan(@JacksonInject StoragePluginRegistry engineRegistry, + @JsonProperty("userName") String userName, + @JsonProperty("entries") List entries, + @JsonProperty("storage") StoragePluginConfig storageConfig, + @JsonProperty("format") FormatPluginConfig formatConfig, + @JsonProperty("columns") List columns, + @JsonProperty("selectionRoot") String selectionRoot, + @JsonProperty("cacheFileRoot") String cacheFileRoot, + @JsonProperty("filter") LogicalExpression filter) throws IOException, ExecutionSetupException { + super(ImpersonationUtil.resolveUserName(userName), columns, entries, filter); Preconditions.checkNotNull(storageConfig); Preconditions.checkNotNull(formatConfig); this.formatPlugin = (ParquetFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatConfig); Preconditions.checkNotNull(formatPlugin); this.fs = ImpersonationUtil.createFileSystem(getUserName(), formatPlugin.getFsConf()); this.formatConfig = formatPlugin.getConfig(); - this.entries = entries; this.selectionRoot = selectionRoot; this.cacheFileRoot = cacheFileRoot; - this.filter = filter; this.metaContext = new MetadataContext(); init(); } - public ParquetGroupScan( // - String userName, - FileSelection selection, // - ParquetFormatPlugin formatPlugin, // - List columns) throws IOException { + public ParquetGroupScan(String userName, + FileSelection selection, + ParquetFormatPlugin formatPlugin, + List columns) throws IOException { this(userName, selection, formatPlugin, columns, ValueExpressions.BooleanExpression.TRUE); } - public ParquetGroupScan( // - String userName, - FileSelection selection, // - ParquetFormatPlugin formatPlugin, // - List columns, - LogicalExpression filter) // - throws IOException { - super(userName); + public ParquetGroupScan(String userName, + FileSelection selection, + ParquetFormatPlugin formatPlugin, + List columns, + LogicalExpression filter) throws IOException { + super(userName, columns, new ArrayList<>(), filter); + this.formatPlugin = formatPlugin; - this.columns = columns; this.formatConfig = formatPlugin.getConfig(); this.fs = ImpersonationUtil.createFileSystem(userName, formatPlugin.getFsConf()); - this.selectionRoot = selection.getSelectionRoot(); this.cacheFileRoot = selection.getCacheFileRoot(); MetadataContext metadataContext = selection.getMetaContext(); this.metaContext = metadataContext != null ? metadataContext : new MetadataContext(); - final FileSelection fileSelection = expandIfNecessary(selection); - - this.entries = Lists.newArrayList(); - + FileSelection fileSelection = expandIfNecessary(selection); if (fileSelection != null) { if (checkForInitializingEntriesWithSelectionRoot()) { // The fully expanded list is already stored as part of the fileSet @@ -220,41 +130,20 @@ public ParquetGroupScan( // } } - this.filter = filter; - init(); } } - /* - * This is used to clone another copy of the group scan. - */ private ParquetGroupScan(ParquetGroupScan that) { this(that, null); } - /* - * This is used to clone another copy of the group scan, but keep the metadata caching info from the file selection - */ private ParquetGroupScan(ParquetGroupScan that, FileSelection selection) { super(that); - this.columns = that.columns == null ? null : Lists.newArrayList(that.columns); - this.endpointAffinities = that.endpointAffinities == null ? null : Lists.newArrayList(that.endpointAffinities); - this.entries = that.entries == null ? null : Lists.newArrayList(that.entries); this.formatConfig = that.formatConfig; this.formatPlugin = that.formatPlugin; this.fs = that.fs; - this.mappings = that.mappings == null ? null : ArrayListMultimap.create(that.mappings); - this.rowCount = that.rowCount; - this.rowGroupInfos = that.rowGroupInfos == null ? null : Lists.newArrayList(that.rowGroupInfos); this.selectionRoot = that.selectionRoot; - this.columnValueCounts = that.columnValueCounts == null ? null : new HashMap<>(that.columnValueCounts); - this.partitionColTypeMap = that.partitionColTypeMap == null ? null : new HashMap<>(that.partitionColTypeMap); - this.partitionValueMap = that.partitionValueMap == null ? null : new HashMap<>(that.partitionValueMap); - this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet); - this.usedMetadataCache = that.usedMetadataCache; - this.parquetTableMetadata = that.parquetTableMetadata; - this.filter = that.filter; if (selection != null) { this.cacheFileRoot = selection.getCacheFileRoot(); MetadataContext metaContext = selection.getMetaContext(); @@ -263,542 +152,206 @@ private ParquetGroupScan(ParquetGroupScan that, FileSelection selection) { this.cacheFileRoot = that.cacheFileRoot; this.metaContext = that.metaContext; } + this.usedMetadataCache = that.usedMetadataCache; } - /** - * expands the selection's folders if metadata cache is found for the selection root.
- * If the selection has already been expanded or no metadata cache was found, does nothing - * - * @param selection actual selection before expansion - * @return new selection after expansion, if no expansion was done returns the input selection - * - * @throws IOException - */ - private FileSelection expandIfNecessary(FileSelection selection) throws IOException { - if (selection.isExpandedFully()) { - return selection; - } - - // 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 - if (selection.isExpandedPartial()) { - logger.error("'{}' metadata file does not exist, but metadata directories cache file is present", metaFilePath); - metaContext.setMetadataCacheCorrupted(true); - } - - return selection; - } - - return expandSelectionFromMetadataCache(selection, metaFilePath); - } - - /** - * For two cases the entries should be initialized with just the selection root instead of the fully expanded list: - *
    - *
  • When metadata caching is corrupted (to use correct file selection) - *
  • Metadata caching is correct and used, but pruning was not applicable or was attempted and nothing was pruned - * (to reduce overhead in parquet group scan). - *
- * - * @return true if entries should be initialized with selection root, false otherwise - */ - private boolean checkForInitializingEntriesWithSelectionRoot() { - // TODO: at some point we should examine whether the list of entries is absolutely needed. - return metaContext.isMetadataCacheCorrupted() || (parquetTableMetadata != null && - (metaContext.getPruneStatus() == PruneStatus.NOT_STARTED || metaContext.getPruneStatus() == PruneStatus.NOT_PRUNED)); - } - - public List getEntries() { - return entries; - } - + // getters for serialization / deserialization start @JsonProperty("format") public ParquetFormatConfig getFormatConfig() { - return this.formatConfig; + return formatConfig; } @JsonProperty("storage") public StoragePluginConfig getEngineConfig() { - return this.formatPlugin.getStorageConfig(); + return formatPlugin.getStorageConfig(); } + @JsonProperty public String getSelectionRoot() { return selectionRoot; } - public Set getFileSet() { - return fileSet; + @JsonProperty + public String getCacheFileRoot() { + return cacheFileRoot; } + // getters for serialization / deserialization end - public LogicalExpression getFilter() { - return this.filter; - } - - public void setFilter(LogicalExpression filter) { - this.filter = filter; + @Override + public ParquetRowGroupScan getSpecificScan(int minorFragmentId) { + return new ParquetRowGroupScan(getUserName(), formatPlugin, getReadEntries(minorFragmentId), columns, selectionRoot, filter); } @Override - public boolean hasFiles() { - return true; + public PhysicalOperator getNewWithChildren(List children) { + Preconditions.checkArgument(children.isEmpty()); + return new ParquetGroupScan(this); } - @JsonIgnore @Override - public Collection getFiles() { - return fileSet; + public GroupScan clone(List columns) { + ParquetGroupScan newScan = new ParquetGroupScan(this); + newScan.columns = columns; + return newScan; } - private Set fileSet; - - @JsonIgnore - // only for partition columns : value is unique for each partition - private Map partitionColTypeMap = Maps.newHashMap(); - - /** - * When reading the very first footer, any column is a potential partition column. So for the first footer, we check - * every column to see if it is single valued, and if so, add it to the list of potential partition columns. For the - * remaining footers, we will not find any new partition columns, but we may discover that what was previously a - * potential partition column now no longer qualifies, so it needs to be removed from the list. - * @return whether column is a potential partition column - */ - private boolean checkForPartitionColumn(ColumnMetadata columnMetadata, boolean first, long rowCount) { - SchemaPath schemaPath = SchemaPath.getCompoundPath(columnMetadata.getName()); - final PrimitiveTypeName primitiveType; - final OriginalType originalType; - int precision = 0; - int scale = 0; - if (this.parquetTableMetadata.hasColumnMetadata()) { - // only ColumnTypeMetadata_v3 stores information about scale and precision - if (parquetTableMetadata instanceof Metadata.ParquetTableMetadata_v3) { - Metadata.ColumnTypeMetadata_v3 columnTypeInfo = ((Metadata.ParquetTableMetadata_v3) parquetTableMetadata) - .getColumnTypeInfo(columnMetadata.getName()); - scale = columnTypeInfo.scale; - precision = columnTypeInfo.precision; - } - primitiveType = this.parquetTableMetadata.getPrimitiveType(columnMetadata.getName()); - originalType = this.parquetTableMetadata.getOriginalType(columnMetadata.getName()); - } else { - primitiveType = columnMetadata.getPrimitiveType(); - originalType = columnMetadata.getOriginalType(); - } - if (first) { - if (hasSingleValue(columnMetadata, rowCount)) { - partitionColTypeMap.put(schemaPath, getType(primitiveType, originalType, scale, precision)); - return true; - } else { - return false; - } - } else { - if (!partitionColTypeMap.keySet().contains(schemaPath)) { - return false; - } else { - if (!hasSingleValue(columnMetadata, rowCount)) { - partitionColTypeMap.remove(schemaPath); - return false; - } - if (!getType(primitiveType, originalType, scale, precision).equals(partitionColTypeMap.get(schemaPath))) { - partitionColTypeMap.remove(schemaPath); - return false; - } - } - } - return true; + @Override + public ParquetGroupScan clone(FileSelection selection) throws IOException { + ParquetGroupScan newScan = new ParquetGroupScan(this, selection); + newScan.modifyFileSelection(selection); + newScan.init(); + return newScan; } - /** - * Builds major type using given {@code OriginalType originalType} or {@code PrimitiveTypeName type}. - * For DECIMAL will be returned major type with scale and precision. - * - * @param type parquet primitive type - * @param originalType parquet original type - * @param scale type scale (used for DECIMAL type) - * @param precision type precision (used for DECIMAL type) - * @return major type - */ - public static MajorType getType(PrimitiveTypeName type, OriginalType originalType, int scale, int precision) { - if (originalType != null) { - switch (originalType) { - case DECIMAL: - return Types.withScaleAndPrecision(MinorType.DECIMAL18, TypeProtos.DataMode.OPTIONAL, scale, precision); - case DATE: - return Types.optional(MinorType.DATE); - case TIME_MILLIS: - return Types.optional(MinorType.TIME); - case TIMESTAMP_MILLIS: - return Types.optional(MinorType.TIMESTAMP); - case UTF8: - return Types.optional(MinorType.VARCHAR); - case UINT_8: - return Types.optional(MinorType.UINT1); - case UINT_16: - return Types.optional(MinorType.UINT2); - case UINT_32: - return Types.optional(MinorType.UINT4); - case UINT_64: - return Types.optional(MinorType.UINT8); - case INT_8: - return Types.optional(MinorType.TINYINT); - case INT_16: - return Types.optional(MinorType.SMALLINT); - case INTERVAL: - return Types.optional(MinorType.INTERVAL); - } - } + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("ParquetGroupScan ["); + builder.append("entries=").append(entries); + builder.append(", selectionRoot=").append(selectionRoot); + builder.append(", numFiles=").append(getEntries().size()); + builder.append(", numRowGroups=").append(rowGroupInfos.size()); + builder.append(", usedMetadataFile=").append(usedMetadataCache); - switch (type) { - case BOOLEAN: - return Types.optional(MinorType.BIT); - case INT32: - return Types.optional(MinorType.INT); - case INT64: - return Types.optional(MinorType.BIGINT); - case FLOAT: - return Types.optional(MinorType.FLOAT4); - case DOUBLE: - return Types.optional(MinorType.FLOAT8); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - case INT96: - return Types.optional(MinorType.VARBINARY); - default: - // Should never hit this - throw new UnsupportedOperationException("Unsupported type:" + type); + String filterString = getFilterString(); + if (!filterString.isEmpty()) { + builder.append(", filter=").append(filterString); } - } - /** - * Checks that the column chunk has a single value. - * - * @param columnChunkMetaData metadata to check - * @param rowCount rows count in column chunk - * @return true if column has single value - */ - private boolean hasSingleValue(ColumnMetadata columnChunkMetaData, long rowCount) { - // ColumnMetadata will have a non-null value iff the minValue and the maxValue for the - // rowgroup are the same - return (columnChunkMetaData != null) && (columnChunkMetaData.hasSingleValue(rowCount)); - } - - @Override public void modifyFileSelection(FileSelection selection) { - entries.clear(); - fileSet = Sets.newHashSet(); - for (String fileName : selection.getFiles()) { - entries.add(new ReadEntryWithPath(fileName)); - fileSet.add(fileName); + if (usedMetadataCache) { + // For EXPLAIN, remove the URI prefix from cacheFileRoot. If cacheFileRoot is null, we + // would have read the cache file from selectionRoot + String cacheFileRootString = (cacheFileRoot == null) ? + Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)).toString() : + Path.getPathWithoutSchemeAndAuthority(new Path(cacheFileRoot)).toString(); + builder.append(", cacheFileRoot=").append(cacheFileRootString); } - List newRowGroupList = Lists.newArrayList(); - for (RowGroupInfo rowGroupInfo : rowGroupInfos) { - if (fileSet.contains(rowGroupInfo.getPath())) { - newRowGroupList.add(rowGroupInfo); - } - } - this.rowGroupInfos = newRowGroupList; - } + builder.append(", columns=").append(columns); + builder.append("]"); - public MajorType getTypeForColumn(SchemaPath schemaPath) { - return partitionColTypeMap.get(schemaPath); + return builder.toString(); } - // Map from file names to maps of column name to partition value mappings - private Map> partitionValueMap = Maps.newHashMap(); - - public void populatePruningVector(ValueVector v, int index, SchemaPath column, String file) { - String f = Path.getPathWithoutSchemeAndAuthority(new Path(file)).toString(); - MajorType majorType = getTypeForColumn(column); - MinorType type = majorType.getMinorType(); - switch (type) { - case BIT: { - NullableBitVector bitVector = (NullableBitVector) v; - Boolean value = (Boolean) partitionValueMap.get(f).get(column); - if (value == null) { - bitVector.getMutator().setNull(index); - } else { - bitVector.getMutator().setSafe(index, value ? 1 : 0); - } - return; - } - case INT: { - NullableIntVector intVector = (NullableIntVector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - intVector.getMutator().setNull(index); - } else { - intVector.getMutator().setSafe(index, value); - } - return; - } - case SMALLINT: { - NullableSmallIntVector smallIntVector = (NullableSmallIntVector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - smallIntVector.getMutator().setNull(index); - } else { - smallIntVector.getMutator().setSafe(index, value.shortValue()); - } - return; - } - case TINYINT: { - NullableTinyIntVector tinyIntVector = (NullableTinyIntVector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - tinyIntVector.getMutator().setNull(index); - } else { - tinyIntVector.getMutator().setSafe(index, value.byteValue()); - } - return; - } - case UINT1: { - NullableUInt1Vector intVector = (NullableUInt1Vector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - intVector.getMutator().setNull(index); - } else { - intVector.getMutator().setSafe(index, value.byteValue()); - } - return; - } - case UINT2: { - NullableUInt2Vector intVector = (NullableUInt2Vector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - intVector.getMutator().setNull(index); - } else { - intVector.getMutator().setSafe(index, (char) value.shortValue()); - } - return; - } - case UINT4: { - NullableUInt4Vector intVector = (NullableUInt4Vector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - intVector.getMutator().setNull(index); - } else { - intVector.getMutator().setSafe(index, value); - } - return; - } - case BIGINT: { - NullableBigIntVector bigIntVector = (NullableBigIntVector) v; - Long value = (Long) partitionValueMap.get(f).get(column); - if (value == null) { - bigIntVector.getMutator().setNull(index); - } else { - bigIntVector.getMutator().setSafe(index, value); - } - return; - } - case FLOAT4: { - NullableFloat4Vector float4Vector = (NullableFloat4Vector) v; - Float value = (Float) partitionValueMap.get(f).get(column); - if (value == null) { - float4Vector.getMutator().setNull(index); - } else { - float4Vector.getMutator().setSafe(index, value); - } - return; - } - case FLOAT8: { - NullableFloat8Vector float8Vector = (NullableFloat8Vector) v; - Double value = (Double) partitionValueMap.get(f).get(column); - if (value == null) { - float8Vector.getMutator().setNull(index); - } else { - float8Vector.getMutator().setSafe(index, value); - } - return; - } - case VARBINARY: { - NullableVarBinaryVector varBinaryVector = (NullableVarBinaryVector) v; - Object s = partitionValueMap.get(f).get(column); - byte[] bytes; - if (s == null) { - varBinaryVector.getMutator().setNull(index); - return; - } else { - bytes = getBytes(type, s); - } - varBinaryVector.getMutator().setSafe(index, bytes, 0, bytes.length); - return; - } - case DECIMAL18: { - NullableDecimal18Vector decimalVector = (NullableDecimal18Vector) v; - Object s = partitionValueMap.get(f).get(column); - byte[] bytes; - if (s == null) { - decimalVector.getMutator().setNull(index); - return; - } else if (s instanceof Integer) { - long value = DecimalUtility.getBigDecimalFromPrimitiveTypes( - (Integer) s, - majorType.getScale(), - majorType.getPrecision()).longValue(); - decimalVector.getMutator().setSafe(index, value); - return; - } else if (s instanceof Long) { - long value = DecimalUtility.getBigDecimalFromPrimitiveTypes( - (Long) s, - majorType.getScale(), - majorType.getPrecision()).longValue(); - decimalVector.getMutator().setSafe(index, value); - return; - } else { - bytes = getBytes(type, s); - } - long value = DecimalUtility.getBigDecimalFromByteArray(bytes, 0, bytes.length, majorType.getScale()).longValue(); - decimalVector.getMutator().setSafe(index, value); - return; + // overridden protected methods block start + @Override + protected void initInternal() throws IOException { + FileSystem processUserFileSystem = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf()); + Path metaPath = null; + if (entries.size() == 1 && parquetTableMetadata == null) { + Path p = Path.getPathWithoutSchemeAndAuthority(new Path(entries.get(0).getPath())); + if (fs.isDirectory(p)) { + // Using the metadata file makes sense when querying a directory; otherwise + // if querying a single file we can look up the metadata directly from the file + metaPath = new Path(p, Metadata.METADATA_FILENAME); } - case DATE: { - NullableDateVector dateVector = (NullableDateVector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - dateVector.getMutator().setNull(index); - } else { - dateVector.getMutator().setSafe(index, value * (long) DateTimeConstants.MILLIS_PER_DAY); + if (!metaContext.isMetadataCacheCorrupted() && metaPath != null && fs.exists(metaPath)) { + parquetTableMetadata = Metadata.readBlockMeta(processUserFileSystem, metaPath, metaContext, formatConfig); + if (parquetTableMetadata != null) { + usedMetadataCache = true; } - return; } - case TIME: { - NullableTimeVector timeVector = (NullableTimeVector) v; - Integer value = (Integer) partitionValueMap.get(f).get(column); - if (value == null) { - timeVector.getMutator().setNull(index); - } else { - timeVector.getMutator().setSafe(index, value); - } - return; + if (!usedMetadataCache) { + parquetTableMetadata = Metadata.getParquetTableMetadata(processUserFileSystem, p.toString(), formatConfig); } - case TIMESTAMP: { - NullableTimeStampVector timeStampVector = (NullableTimeStampVector) v; - Long value = (Long) partitionValueMap.get(f).get(column); - if (value == null) { - timeStampVector.getMutator().setNull(index); - } else { - timeStampVector.getMutator().setSafe(index, value); + } else { + Path p = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)); + metaPath = new Path(p, Metadata.METADATA_FILENAME); + if (!metaContext.isMetadataCacheCorrupted() && fs.isDirectory(new Path(selectionRoot)) + && fs.exists(metaPath)) { + if (parquetTableMetadata == null) { + parquetTableMetadata = Metadata.readBlockMeta(processUserFileSystem, metaPath, metaContext, formatConfig); } - return; - } - case VARCHAR: { - NullableVarCharVector varCharVector = (NullableVarCharVector) v; - Object s = partitionValueMap.get(f).get(column); - byte[] bytes; - if (s == null) { - varCharVector.getMutator().setNull(index); - return; - } else { - bytes = getBytes(type, s); + if (parquetTableMetadata != null) { + usedMetadataCache = true; + if (fileSet != null) { + parquetTableMetadata = removeUnneededRowGroups(parquetTableMetadata); + } } - varCharVector.getMutator().setSafe(index, bytes, 0, bytes.length); - return; } - case INTERVAL: { - NullableIntervalVector intervalVector = (NullableIntervalVector) v; - Object s = partitionValueMap.get(f).get(column); - byte[] bytes; - if (s == null) { - intervalVector.getMutator().setNull(index); - return; - } else { - bytes = getBytes(type, s); + if (!usedMetadataCache) { + final List fileStatuses = new ArrayList<>(); + for (ReadEntryWithPath entry : entries) { + fileStatuses.addAll( + DrillFileSystemUtil.listFiles(fs, Path.getPathWithoutSchemeAndAuthority(new Path(entry.getPath())), true)); } - intervalVector.getMutator().setSafe(index, 1, - ParquetReaderUtility.getIntFromLEBytes(bytes, 0), - ParquetReaderUtility.getIntFromLEBytes(bytes, 4), - ParquetReaderUtility.getIntFromLEBytes(bytes, 8)); - return; + + Map statusMap = fileStatuses.stream() + .collect( + Collectors.toMap( + Function.identity(), + s -> processUserFileSystem, + (oldFs, newFs) -> newFs, + LinkedHashMap::new)); + + parquetTableMetadata = Metadata.getParquetTableMetadata(statusMap, formatConfig); } - default: - throw new UnsupportedOperationException("Unsupported type: " + type); } } - /** - * Returns the sequence of bytes received from {@code Object source}. - * - * @param type the column type - * @param source the source of the bytes sequence - * @return bytes sequence obtained from {@code Object source} - */ - private byte[] getBytes(MinorType type, Object source) { - byte[] bytes; - if (source instanceof Binary) { - bytes = ((Binary) source).getBytes(); - } else if (source instanceof byte[]) { - bytes = (byte[]) source; - } else { - throw new UnsupportedOperationException("Unable to create column data for type: " + type); - } - return bytes; + @Override + protected AbstractParquetGroupScan cloneWithFileSelection(Collection filePaths) throws IOException { + FileSelection newSelection = new FileSelection(null, new ArrayList<>(filePaths), getSelectionRoot(), cacheFileRoot, false); + return clone(newSelection); } - public static class RowGroupInfo extends ReadEntryFromHDFS implements CompleteWork, FileWork { - - private EndpointByteMap byteMap; - private int rowGroupIndex; - private List columns; - private long rowCount; // rowCount = -1 indicates to include all rows. - private long numRecordsToRead; - - @JsonCreator - public RowGroupInfo(@JsonProperty("path") String path, @JsonProperty("start") long start, - @JsonProperty("length") long length, @JsonProperty("rowGroupIndex") int rowGroupIndex, long rowCount) { - super(path, start, length); - this.rowGroupIndex = rowGroupIndex; - this.rowCount = rowCount; - this.numRecordsToRead = rowCount; - } - - public RowGroupReadEntry getRowGroupReadEntry() { - return new RowGroupReadEntry(this.getPath(), this.getStart(), this.getLength(), - this.rowGroupIndex, this.getNumRecordsToRead()); - } - - public int getRowGroupIndex() { - return this.rowGroupIndex; - } - - @Override - public int compareTo(CompleteWork o) { - return Long.compare(getTotalBytes(), o.getTotalBytes()); - } - - @Override - public long getTotalBytes() { - return this.getLength(); - } - - @Override - public EndpointByteMap getByteMap() { - return byteMap; - } + @Override + protected Collection getDrillbits() { + return formatPlugin.getContext().getBits(); + } - public long getNumRecordsToRead() { - return numRecordsToRead; - } + @Override + protected boolean supportsFileImplicitColumns() { + return selectionRoot != null; + } - public void setNumRecordsToRead(long numRecords) { - numRecordsToRead = numRecords; - } + @Override + protected List getPartitionValues(RowGroupInfo rowGroupInfo) { + return ColumnExplorer.listPartitionValues(rowGroupInfo.getPath(), selectionRoot); + } + // overridden protected methods block end - public void setEndpointByteMap(EndpointByteMap byteMap) { - this.byteMap = byteMap; + // private methods block start + /** + * Expands the selection's folders if metadata cache is found for the selection root.
+ * If the selection has already been expanded or no metadata cache was found, does nothing + * + * @param selection actual selection before expansion + * @return new selection after expansion, if no expansion was done returns the input selection + */ + private FileSelection expandIfNecessary(FileSelection selection) throws IOException { + if (selection.isExpandedFully()) { + return selection; } - public long getRowCount() { - return rowCount; - } + // 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 + if (selection.isExpandedPartial()) { + logger.error("'{}' metadata file does not exist, but metadata directories cache file is present", metaFilePath); + metaContext.setMetadataCacheCorrupted(true); + } - public List getColumns() { - return columns; + return selection; } - public void setColumns(List columns) { - this.columns = columns; - } + return expandSelectionFromMetadataCache(selection, metaFilePath); + } + /** + * For two cases the entries should be initialized with just the selection root instead of the fully expanded list: + *
    + *
  • When metadata caching is corrupted (to use correct file selection) + *
  • Metadata caching is correct and used, but pruning was not applicable or was attempted and nothing was pruned + * (to reduce overhead in parquet group scan). + *
+ * + * @return true if entries should be initialized with selection root, false otherwise + */ + private boolean checkForInitializingEntriesWithSelectionRoot() { + // TODO: at some point we should examine whether the list of entries is absolutely needed. + return metaContext.isMetadataCacheCorrupted() || (parquetTableMetadata != null && + (metaContext.getPruneStatus() == PruneStatus.NOT_STARTED || metaContext.getPruneStatus() == PruneStatus.NOT_PRUNED)); } /** @@ -810,17 +363,16 @@ public void setColumns(List columns) { * @param metaFilePath metadata cache file path * @return file selection read from cache * - * @throws IOException * @throws UserException when the updated selection is empty, this happens if the user selects an empty folder. */ - private FileSelection - expandSelectionFromMetadataCache(FileSelection selection, Path metaFilePath) throws IOException { + private FileSelection expandSelectionFromMetadataCache(FileSelection selection, Path metaFilePath) throws IOException { // get the metadata for the root directory by reading the metadata file // parquetTableMetadata contains the metadata for all files in the selection root folder, but we need to make sure // we only select the files that are part of selection (by setting fileSet appropriately) // get (and set internal field) the metadata for the directory by reading the metadata file - parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath, metaContext, formatConfig); + FileSystem processUserFileSystem = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fs.getConf()); + parquetTableMetadata = Metadata.readBlockMeta(processUserFileSystem, metaFilePath, metaContext, formatConfig); if (ignoreExpandingSelection(parquetTableMetadata)) { return selection; } @@ -837,7 +389,7 @@ public void setColumns(List columns) { 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()) { + for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { fileSet.add(file.getPath()); } @@ -850,7 +402,7 @@ public void setColumns(List columns) { // list of files from the metadata cache file that is present in the cacheFileRoot directory and populate // the fileSet. However, this is *not* the final list of files that will be scanned in execution since the // second phase of partition pruning will apply on the files and modify the file selection appropriately. - for (Metadata.ParquetFileMetadata file : this.parquetTableMetadata.getFiles()) { + for (ParquetFileMetadata file : this.parquetTableMetadata.getFiles()) { fileSet.add(file.getPath()); } } @@ -861,11 +413,11 @@ public void setColumns(List columns) { if (status.isDirectory()) { //TODO [DRILL-4496] read the metadata cache files in parallel final Path metaPath = new Path(cacheFileRoot, Metadata.METADATA_FILENAME); - final Metadata.ParquetTableMetadataBase metadata = Metadata.readBlockMeta(fs, metaPath, metaContext, formatConfig); + final ParquetTableMetadataBase metadata = Metadata.readBlockMeta(processUserFileSystem, metaPath, metaContext, formatConfig); if (ignoreExpandingSelection(metadata)) { return selection; } - for (Metadata.ParquetFileMetadata file : metadata.getFiles()) { + for (ParquetFileMetadata file : metadata.getFiles()) { fileSet.add(file.getPath()); } } else { @@ -881,7 +433,7 @@ public void setColumns(List columns) { return null; } - List fileNames = Lists.newArrayList(fileSet); + List fileNames = new ArrayList<>(fileSet); // 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 @@ -902,141 +454,8 @@ public void setColumns(List columns) { return newSelection; } - private void init() throws IOException { - Path metaPath = null; - if (entries.size() == 1 && parquetTableMetadata == null) { - Path p = Path.getPathWithoutSchemeAndAuthority(new Path(entries.get(0).getPath())); - if (fs.isDirectory(p)) { - // Using the metadata file makes sense when querying a directory; otherwise - // if querying a single file we can look up the metadata directly from the file - metaPath = new Path(p, Metadata.METADATA_FILENAME); - } - if (!metaContext.isMetadataCacheCorrupted() && metaPath != null && fs.exists(metaPath)) { - parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath, metaContext, formatConfig); - if (parquetTableMetadata != null) { - usedMetadataCache = true; - } - } - if (!usedMetadataCache) { - parquetTableMetadata = Metadata.getParquetTableMetadata(fs, p.toString(), formatConfig); - } - } else { - Path p = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot)); - metaPath = new Path(p, Metadata.METADATA_FILENAME); - if (!metaContext.isMetadataCacheCorrupted() && fs.isDirectory(new Path(selectionRoot)) - && fs.exists(metaPath)) { - if (parquetTableMetadata == null) { - parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath, metaContext, formatConfig); - } - if (parquetTableMetadata != null) { - usedMetadataCache = true; - if (fileSet != null) { - parquetTableMetadata = removeUnneededRowGroups(parquetTableMetadata); - } - } - } - if (!usedMetadataCache) { - final List fileStatuses = Lists.newArrayList(); - for (ReadEntryWithPath entry : entries) { - fileStatuses.addAll(DrillFileSystemUtil.listFiles(fs, Path.getPathWithoutSchemeAndAuthority(new Path(entry.getPath())), true)); - } - parquetTableMetadata = Metadata.getParquetTableMetadata(fs, fileStatuses, formatConfig); - } - } - - if (fileSet == null) { - fileSet = Sets.newHashSet(); - for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { - fileSet.add(file.getPath()); - } - } - - Map hostEndpointMap = Maps.newHashMap(); - - for (DrillbitEndpoint endpoint : formatPlugin.getContext().getBits()) { - hostEndpointMap.put(endpoint.getAddress(), endpoint); - } - - rowGroupInfos = Lists.newArrayList(); - for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { - int rgIndex = 0; - for (RowGroupMetadata rg : file.getRowGroups()) { - RowGroupInfo rowGroupInfo = - new RowGroupInfo(file.getPath(), rg.getStart(), rg.getLength(), rgIndex, rg.getRowCount()); - EndpointByteMap endpointByteMap = new EndpointByteMapImpl(); - for (String host : rg.getHostAffinity().keySet()) { - if (hostEndpointMap.containsKey(host)) { - endpointByteMap - .add(hostEndpointMap.get(host), (long) (rg.getHostAffinity().get(host) * rg.getLength())); - } - } - rowGroupInfo.setEndpointByteMap(endpointByteMap); - rowGroupInfo.setColumns(rg.getColumns()); - rgIndex++; - rowGroupInfos.add(rowGroupInfo); - } - } - - this.endpointAffinities = AffinityCreator.getAffinityMap(rowGroupInfos); - updatePartitionColTypeMap(); - } - - private void updatePartitionColTypeMap() { - columnValueCounts = Maps.newHashMap(); - this.rowCount = 0; - boolean first = true; - for (RowGroupInfo rowGroup : this.rowGroupInfos) { - long rowCount = rowGroup.getRowCount(); - for (ColumnMetadata column : rowGroup.getColumns()) { - SchemaPath schemaPath = SchemaPath.getCompoundPath(column.getName()); - Long previousCount = columnValueCounts.get(schemaPath); - if (previousCount != null) { - if (previousCount != GroupScan.NO_COLUMN_STATS) { - if (column.getNulls() != null) { - Long newCount = rowCount - column.getNulls(); - columnValueCounts.put(schemaPath, columnValueCounts.get(schemaPath) + newCount); - } - } - } else { - if (column.getNulls() != null) { - Long newCount = rowCount - column.getNulls(); - columnValueCounts.put(schemaPath, newCount); - } else { - columnValueCounts.put(schemaPath, GroupScan.NO_COLUMN_STATS); - } - } - boolean partitionColumn = checkForPartitionColumn(column, first, rowCount); - if (partitionColumn) { - Map map = partitionValueMap.get(rowGroup.getPath()); - if (map == null) { - map = Maps.newHashMap(); - partitionValueMap.put(rowGroup.getPath(), map); - } - Object value = map.get(schemaPath); - Object currentValue = column.getMaxValue(); - if (value != null) { - if (value != currentValue) { - partitionColTypeMap.remove(schemaPath); - } - } else { - // the value of a column with primitive type can not be null, - // so checks that there are really null value and puts it to the map - if (rowCount == column.getNulls()) { - map.put(schemaPath, null); - } else { - map.put(schemaPath, currentValue); - } - } - } else { - partitionColTypeMap.remove(schemaPath); - } - } - this.rowCount += rowGroup.getRowCount(); - first = false; - } - } private ParquetTableMetadataBase removeUnneededRowGroups(ParquetTableMetadataBase parquetTableMetadata) { - List newFileMetadataList = Lists.newArrayList(); + List newFileMetadataList = new ArrayList<>(); for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { if (fileSet.contains(file.getPath())) { newFileMetadataList.add(file); @@ -1048,284 +467,6 @@ private ParquetTableMetadataBase removeUnneededRowGroups(ParquetTableMetadataBas return metadata; } - /** - * Calculates the affinity each endpoint has for this scan, by adding up the affinity each endpoint has for each - * rowGroup - * - * @return a list of EndpointAffinity objects - */ - @Override - public List getOperatorAffinity() { - return this.endpointAffinities; - } - - @Override - public void applyAssignments(List incomingEndpoints) throws PhysicalOperatorSetupException { - - this.mappings = AssignmentCreator.getMappings(incomingEndpoints, rowGroupInfos); - } - - @Override public ParquetRowGroupScan getSpecificScan(int minorFragmentId) { - assert minorFragmentId < mappings.size() : String - .format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", - mappings.size(), minorFragmentId); - - List rowGroupsForMinor = mappings.get(minorFragmentId); - - Preconditions.checkArgument(!rowGroupsForMinor.isEmpty(), - String.format("MinorFragmentId %d has no read entries assigned", minorFragmentId)); - - return new ParquetRowGroupScan( - getUserName(), formatPlugin, convertToReadEntries(rowGroupsForMinor), columns, selectionRoot, filter); - } - - private List convertToReadEntries(List rowGroups) { - List entries = Lists.newArrayList(); - for (RowGroupInfo rgi : rowGroups) { - RowGroupReadEntry entry = new RowGroupReadEntry(rgi.getPath(), rgi.getStart(), rgi.getLength(), rgi.getRowGroupIndex(), rgi.getNumRecordsToRead()); - entries.add(entry); - } - return entries; - } - - @Override - public int getMaxParallelizationWidth() { - return rowGroupInfos.size(); - } - - public List getColumns() { - return columns; - } - - @Override - public ScanStats getScanStats() { - int columnCount = columns == null ? 20 : columns.size(); - return new ScanStats(GroupScanProperty.EXACT_ROW_COUNT, rowCount, 1, rowCount * columnCount); - } - - @Override - @JsonIgnore - public PhysicalOperator getNewWithChildren(List children) { - Preconditions.checkArgument(children.isEmpty()); - return new ParquetGroupScan(this); - } - - @Override - public String getDigest() { - return toString(); - } - - @Override - public String toString() { - String cacheFileString = ""; - if (usedMetadataCache) { - // 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; - } - final String filterStr = filter == null || filter.equals(ValueExpressions.BooleanExpression.TRUE) ? "" : ", filter=" + ExpressionStringBuilder.toString(this.filter); - - return "ParquetGroupScan [entries=" + entries - + ", selectionRoot=" + selectionRoot - + ", numFiles=" + getEntries().size() - + ", numRowGroups=" + rowGroupInfos.size() - + ", usedMetadataFile=" + usedMetadataCache - + filterStr - + cacheFileString - + ", columns=" + columns - + "]"; - } - - @Override - public GroupScan clone(List columns) { - ParquetGroupScan newScan = new ParquetGroupScan(this); - newScan.columns = columns; - return newScan; - } - - // Based on maxRecords to read for the scan, - // figure out how many rowGroups to read and update number of records to read for each of them. - // Returns total number of rowGroups to read. - private int updateRowGroupInfo(long maxRecords) { - long count = 0; - int index = 0; - for (RowGroupInfo rowGroupInfo : rowGroupInfos) { - long rowCount = rowGroupInfo.getRowCount(); - if (count + rowCount <= maxRecords) { - count += rowCount; - rowGroupInfo.setNumRecordsToRead(rowCount); - index++; - continue; - } else if (count < maxRecords) { - rowGroupInfo.setNumRecordsToRead(maxRecords - count); - index++; - } - break; - } - - return index; - } - - @Override - public ParquetGroupScan clone(FileSelection selection) throws IOException { - ParquetGroupScan newScan = new ParquetGroupScan(this, selection); - newScan.modifyFileSelection(selection); - newScan.init(); - return newScan; - } - - public ParquetGroupScan clone(FileSelection selection, long maxRecords) throws IOException { - ParquetGroupScan newScan = clone(selection); - newScan.updateRowGroupInfo(maxRecords); - return newScan; - } - - @Override - public boolean supportsLimitPushdown() { - return true; - } - - @Override - public GroupScan applyLimit(int maxRecords) { - Preconditions.checkArgument(rowGroupInfos.size() >= 0); - - maxRecords = Math.max(maxRecords, 1); // Make sure it request at least 1 row -> 1 rowGroup. - // further optimization : minimize # of files chosen, or the affinity of files chosen. - - // Calculate number of rowGroups to read based on maxRecords and update - // number of records to read for each of those rowGroups. - int index = updateRowGroupInfo(maxRecords); - - Set fileNames = Sets.newHashSet(); // HashSet keeps a fileName unique. - for (RowGroupInfo rowGroupInfo : rowGroupInfos.subList(0, index)) { - fileNames.add(rowGroupInfo.getPath()); - } - - // If there is no change in fileSet, no need to create new groupScan. - if (fileNames.size() == fileSet.size() ) { - // There is no reduction of rowGroups. Return the original groupScan. - logger.debug("applyLimit() does not apply!"); - return null; - } - - try { - 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, maxRecords); - } catch (IOException e) { - logger.warn("Could not apply rowcount based prune due to Exception : {}", e); - return null; - } - } - - @Override - @JsonIgnore - public boolean canPushdownProjects(List columns) { - return true; - } - - /** - * Return column value count for the specified column. If does not contain such column, return 0. - */ - @Override - public long getColumnValueCount(SchemaPath column) { - return columnValueCounts.containsKey(column) ? columnValueCounts.get(column) : 0; - } - - @Override - public List getPartitionColumns() { - return new ArrayList<>(partitionColTypeMap.keySet()); - } - - public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities, - FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) { - if (rowGroupInfos.size() == 1 || - ! (parquetTableMetadata.isRowGroupPrunable()) || - rowGroupInfos.size() > optionManager.getOption(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD) - ) { - // Stop pruning for 3 cases: - // - 1 single parquet file, - // - metadata does not have proper format to support row group level filter pruning, - // - # of row groups is beyond PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD. - return null; - } - - final Set schemaPathsInExpr = filterExpr.accept(new ParquetRGFilterEvaluator.FieldReferenceFinder(), null); - - final List qualifiedRGs = new ArrayList<>(rowGroupInfos.size()); - Set qualifiedFileNames = Sets.newHashSet(); // HashSet keeps a fileName unique. - - ParquetFilterPredicate filterPredicate = null; - - for (RowGroupInfo rowGroup : rowGroupInfos) { - final ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, this.columns); - Map implicitColValues = columnExplorer.populateImplicitColumns(rowGroup.getPath(), selectionRoot); - - ParquetMetaStatCollector statCollector = new ParquetMetaStatCollector( - parquetTableMetadata, - rowGroup.getColumns(), - implicitColValues); - - Map columnStatisticsMap = statCollector.collectColStat(schemaPathsInExpr); - - if (filterPredicate == null) { - ErrorCollector errorCollector = new ErrorCollectorImpl(); - LogicalExpression materializedFilter = ExpressionTreeMaterializer.materializeFilterExpr( - filterExpr, columnStatisticsMap, errorCollector, functionImplementationRegistry); - - if (errorCollector.hasErrors()) { - logger.error("{} error(s) encountered when materialize filter expression : {}", - errorCollector.getErrorCount(), errorCollector.toErrorString()); - return null; - } - // logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter)); - - Set constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter); - filterPredicate = (ParquetFilterPredicate) ParquetFilterBuilder.buildParquetFilterPredicate( - materializedFilter, constantBoundaries, udfUtilities); - - if (filterPredicate == null) { - return null; - } - } - - if (ParquetRGFilterEvaluator.canDrop(filterPredicate, columnStatisticsMap, rowGroup.getRowCount())) { - continue; - } - - qualifiedRGs.add(rowGroup); - qualifiedFileNames.add(rowGroup.getPath()); // TODO : optimize when 1 file contains m row groups. - } - - - if (qualifiedRGs.size() == rowGroupInfos.size() ) { - // There is no reduction of rowGroups. Return the original groupScan. - logger.debug("applyFilter does not have any pruning!"); - return null; - } else if (qualifiedFileNames.size() == 0) { - logger.warn("All rowgroups have been filtered out. Add back one to get schema from scannner"); - RowGroupInfo rg = rowGroupInfos.iterator().next(); - qualifiedFileNames.add(rg.getPath()); - qualifiedRGs.add(rg); - } - - try { - FileSelection newSelection = new FileSelection(null, Lists.newArrayList(qualifiedFileNames), getSelectionRoot(), cacheFileRoot, false); - logger.info("applyFilter {} reduce parquet rowgroup # from {} to {}", ExpressionStringBuilder.toString(filterExpr), rowGroupInfos.size(), qualifiedRGs.size()); - ParquetGroupScan clonegroupscan = this.clone(newSelection); - clonegroupscan.rowGroupInfos = qualifiedRGs; - clonegroupscan.updatePartitionColTypeMap(); - return clonegroupscan; - - } catch (IOException e) { - logger.warn("Could not apply filter prune due to Exception : {}", e); - return null; - } - } - /** * If metadata is corrupted, ignore expanding selection and reset parquetTableMetadata and fileSet fields * @@ -1341,5 +482,6 @@ private boolean ignoreExpandingSelection(ParquetTableMetadataBase metadata) { } return false; } + // private methods block end } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java new file mode 100644 index 00000000000..5a3e0c2e4be --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet; + +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.exec.physical.base.GroupScan; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ColumnTypeMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTableMetadata_v3; + +/** + * Holds common statistics about data in parquet group scan, + * including information about total row count, columns counts, partition columns. + */ +public class ParquetGroupScanStatistics { + + // map from file names to maps of column name to partition value mappings + private Map> partitionValueMap; + // only for partition columns : value is unique for each partition + private Map partitionColTypeMap; + // total number of non-null value for each column in parquet files + private Map columnValueCounts; + // total number of rows (obtained from parquet footer) + private long rowCount; + + + public ParquetGroupScanStatistics(List rowGroupInfos, ParquetTableMetadataBase parquetTableMetadata) { + collect(rowGroupInfos, parquetTableMetadata); + } + + public ParquetGroupScanStatistics(ParquetGroupScanStatistics that) { + this.partitionValueMap = new HashMap<>(that.partitionValueMap); + this.partitionColTypeMap = new HashMap<>(that.partitionColTypeMap); + this.columnValueCounts = new HashMap<>(that.columnValueCounts); + this.rowCount = that.rowCount; + } + + public long getColumnValueCount(SchemaPath column) { + return columnValueCounts.containsKey(column) ? columnValueCounts.get(column) : 0; + } + + public List getPartitionColumns() { + return new ArrayList<>(partitionColTypeMap.keySet()); + } + + public TypeProtos.MajorType getTypeForColumn(SchemaPath schemaPath) { + return partitionColTypeMap.get(schemaPath); + } + + public long getRowCount() { + return rowCount; + } + + public Object getPartitionValue(String path, SchemaPath column) { + return partitionValueMap.get(path).get(column); + } + + public void collect(List rowGroupInfos, ParquetTableMetadataBase parquetTableMetadata) { + resetHolders(); + boolean first = true; + for (RowGroupInfo rowGroup : rowGroupInfos) { + long rowCount = rowGroup.getRowCount(); + for (ColumnMetadata column : rowGroup.getColumns()) { + SchemaPath schemaPath = SchemaPath.getCompoundPath(column.getName()); + Long previousCount = columnValueCounts.get(schemaPath); + if (previousCount != null) { + if (previousCount != GroupScan.NO_COLUMN_STATS && column.getNulls() != null) { + Long newCount = rowCount - column.getNulls(); + columnValueCounts.put(schemaPath, columnValueCounts.get(schemaPath) + newCount); + } + } else { + if (column.getNulls() != null) { + Long newCount = rowCount - column.getNulls(); + columnValueCounts.put(schemaPath, newCount); + } else { + columnValueCounts.put(schemaPath, GroupScan.NO_COLUMN_STATS); + } + } + boolean partitionColumn = checkForPartitionColumn(column, first, rowCount, parquetTableMetadata); + if (partitionColumn) { + Map map = partitionValueMap.computeIfAbsent(rowGroup.getPath(), key -> new HashMap<>()); + Object value = map.get(schemaPath); + Object currentValue = column.getMaxValue(); + if (value != null) { + if (value != currentValue) { + partitionColTypeMap.remove(schemaPath); + } + } else { + // the value of a column with primitive type can not be null, + // so checks that there are really null value and puts it to the map + if (rowCount == column.getNulls()) { + map.put(schemaPath, null); + } else { + map.put(schemaPath, currentValue); + } + } + } else { + partitionColTypeMap.remove(schemaPath); + } + } + this.rowCount += rowGroup.getRowCount(); + first = false; + } + } + + /** + * Re-init holders eigther during first instance creation or statistics update based on updated list of row groups. + */ + private void resetHolders() { + this.partitionValueMap = new HashMap<>(); + this.partitionColTypeMap = new HashMap<>(); + this.columnValueCounts = new HashMap<>(); + this.rowCount = 0; + } + + /** + * When reading the very first footer, any column is a potential partition column. So for the first footer, we check + * every column to see if it is single valued, and if so, add it to the list of potential partition columns. For the + * remaining footers, we will not find any new partition columns, but we may discover that what was previously a + * potential partition column now no longer qualifies, so it needs to be removed from the list. + * + * @param columnMetadata column metadata + * @param first if columns first appears in row group + * @param rowCount row count + * @param parquetTableMetadata parquet table metadata + * + * @return whether column is a potential partition column + */ + private boolean checkForPartitionColumn(ColumnMetadata columnMetadata, + boolean first, + long rowCount, + ParquetTableMetadataBase parquetTableMetadata) { + SchemaPath schemaPath = SchemaPath.getCompoundPath(columnMetadata.getName()); + final PrimitiveType.PrimitiveTypeName primitiveType; + final OriginalType originalType; + int precision = 0; + int scale = 0; + if (parquetTableMetadata.hasColumnMetadata()) { + // only ColumnTypeMetadata_v3 stores information about scale and precision + if (parquetTableMetadata instanceof ParquetTableMetadata_v3) { + ColumnTypeMetadata_v3 columnTypeInfo = ((ParquetTableMetadata_v3) parquetTableMetadata) + .getColumnTypeInfo(columnMetadata.getName()); + scale = columnTypeInfo.scale; + precision = columnTypeInfo.precision; + } + primitiveType = parquetTableMetadata.getPrimitiveType(columnMetadata.getName()); + originalType = parquetTableMetadata.getOriginalType(columnMetadata.getName()); + } else { + primitiveType = columnMetadata.getPrimitiveType(); + originalType = columnMetadata.getOriginalType(); + } + if (first) { + if (hasSingleValue(columnMetadata, rowCount)) { + partitionColTypeMap.put(schemaPath, ParquetReaderUtility.getType(primitiveType, originalType, scale, precision)); + return true; + } else { + return false; + } + } else { + if (!partitionColTypeMap.keySet().contains(schemaPath)) { + return false; + } else { + if (!hasSingleValue(columnMetadata, rowCount)) { + partitionColTypeMap.remove(schemaPath); + return false; + } + if (!ParquetReaderUtility.getType(primitiveType, originalType, scale, precision).equals(partitionColTypeMap.get(schemaPath))) { + partitionColTypeMap.remove(schemaPath); + return false; + } + } + } + return true; + } + + /** + * Checks that the column chunk has a single value. + * ColumnMetadata will have a non-null value iff the minValue and + * the maxValue for the rowgroup are the same. + * + * @param columnChunkMetaData metadata to check + * @param rowCount rows count in column chunk + * + * @return true if column has single value + */ + private boolean hasSingleValue(ColumnMetadata columnChunkMetaData, long rowCount) { + return (columnChunkMetaData != null) && (columnChunkMetaData.hasSingleValue(rowCount)); + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java index 45fa51d58f8..83ce4d24543 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java @@ -65,7 +65,7 @@ public static RelOptRule getFilterOnProject(OptimizerRulesContext optimizerRules @Override public boolean matches(RelOptRuleCall call) { final ScanPrel scan = call.rel(2); - if (scan.getGroupScan() instanceof ParquetGroupScan) { + if (scan.getGroupScan() instanceof AbstractParquetGroupScan) { return super.matches(call); } return false; @@ -90,7 +90,7 @@ public static StoragePluginOptimizerRule getFilterOnScan(OptimizerRulesContext o @Override public boolean matches(RelOptRuleCall call) { final ScanPrel scan = call.rel(1); - if (scan.getGroupScan() instanceof ParquetGroupScan) { + if (scan.getGroupScan() instanceof AbstractParquetGroupScan) { return super.matches(call); } return false; @@ -114,7 +114,7 @@ private ParquetPushDownFilter(RelOptRuleOperand operand, String id, OptimizerRul } protected void doOnMatch(RelOptRuleCall call, FilterPrel filter, ProjectPrel project, ScanPrel scan) { - ParquetGroupScan groupScan = (ParquetGroupScan) scan.getGroupScan(); + AbstractParquetGroupScan groupScan = (AbstractParquetGroupScan) scan.getGroupScan(); if (groupScan.getFilter() != null && !groupScan.getFilter().equals(ValueExpressions.BooleanExpression.TRUE)) { return; } @@ -152,25 +152,26 @@ protected void doOnMatch(RelOptRuleCall call, FilterPrel filter, ProjectPrel pro LogicalExpression conditionExp = DrillOptiq.toDrill( new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, qualifedPred); - Stopwatch timer = Stopwatch.createStarted(); + + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; final GroupScan newGroupScan = groupScan.applyFilter(conditionExp,optimizerContext, optimizerContext.getFunctionRegistry(), optimizerContext.getPlannerSettings().getOptions()); - logger.info("Took {} ms to apply filter on parquet row groups. ", timer.elapsed(TimeUnit.MILLISECONDS)); + if (timer != null) { + logger.debug("Took {} ms to apply filter on parquet row groups. ", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + } if (newGroupScan == null ) { return; } - final ScanPrel newScanRel = ScanPrel.create(scan, scan.getTraitSet(), newGroupScan, scan.getRowType()); - RelNode inputRel = newScanRel; + RelNode newScan = ScanPrel.create(scan, scan.getTraitSet(), newGroupScan, scan.getRowType());; if (project != null) { - inputRel = project.copy(project.getTraitSet(), ImmutableList.of(inputRel)); + newScan = project.copy(project.getTraitSet(), ImmutableList.of(newScan)); } - - final RelNode newFilter = filter.copy(filter.getTraitSet(), ImmutableList.of(inputRel)); - + final RelNode newFilter = filter.copy(filter.getTraitSet(), ImmutableList.of(newScan)); call.transformTo(newFilter); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java index c2580cb4dc4..9157590698d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java @@ -22,9 +22,12 @@ import org.apache.drill.common.exceptions.UserException; import org.apache.drill.common.expression.PathSegment; import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.common.types.TypeProtos; +import org.apache.drill.common.types.Types; import org.apache.drill.exec.expr.holders.NullableTimeStampHolder; import org.apache.drill.exec.planner.physical.PlannerSettings; import org.apache.drill.exec.server.options.OptionManager; +import org.apache.drill.exec.store.parquet.metadata.MetadataVersion; import org.apache.drill.exec.util.Utilities; import org.apache.drill.exec.work.ExecErrorConstants; import org.apache.parquet.SemanticVersion; @@ -53,6 +56,15 @@ import java.util.Map; import java.util.Set; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V2.ColumnTypeMetadata_v2; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V2.ParquetTableMetadata_v2; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ColumnTypeMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTableMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.RowGroupMetadata; + /** * Utility class where we can capture common logic between the two parquet readers */ @@ -142,7 +154,7 @@ public static int autoCorrectCorruptedDate(int corruptedDate) { return (int) (corruptedDate - CORRECT_CORRUPT_DATE_SHIFT); } - public static void correctDatesInMetadataCache(Metadata.ParquetTableMetadataBase parquetTableMetadata) { + public static void correctDatesInMetadataCache(ParquetTableMetadataBase parquetTableMetadata) { DateCorruptionStatus cacheFileCanContainsCorruptDates = new MetadataVersion(parquetTableMetadata.getMetadataVersion()).compareTo(new MetadataVersion(3, 0)) >= 0 ? DateCorruptionStatus.META_SHOWS_NO_CORRUPTION : DateCorruptionStatus.META_UNCLEAR_TEST_VALUES; @@ -150,19 +162,19 @@ public static void correctDatesInMetadataCache(Metadata.ParquetTableMetadataBase // Looking for the DATE data type of column names in the metadata cache file ("metadata_version" : "v2") String[] names = new String[0]; if (new MetadataVersion(2, 0).equals(new MetadataVersion(parquetTableMetadata.getMetadataVersion()))) { - for (Metadata.ColumnTypeMetadata_v2 columnTypeMetadata : - ((Metadata.ParquetTableMetadata_v2) parquetTableMetadata).columnTypeInfo.values()) { + for (ColumnTypeMetadata_v2 columnTypeMetadata : + ((ParquetTableMetadata_v2) parquetTableMetadata).columnTypeInfo.values()) { if (OriginalType.DATE.equals(columnTypeMetadata.originalType)) { names = columnTypeMetadata.name; } } } - for (Metadata.ParquetFileMetadata file : parquetTableMetadata.getFiles()) { + for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { // Drill has only ever written a single row group per file, only need to correct the statistics // on the first row group - Metadata.RowGroupMetadata rowGroupMetadata = file.getRowGroups().get(0); + RowGroupMetadata rowGroupMetadata = file.getRowGroups().get(0); Long rowCount = rowGroupMetadata.getRowCount(); - for (Metadata.ColumnMetadata columnMetadata : rowGroupMetadata.getColumns()) { + for (ColumnMetadata columnMetadata : rowGroupMetadata.getColumns()) { // Setting Min/Max values for ParquetTableMetadata_v1 if (new MetadataVersion(1, 0).equals(new MetadataVersion(parquetTableMetadata.getMetadataVersion()))) { OriginalType originalType = columnMetadata.getOriginalType(); @@ -192,15 +204,15 @@ else if (new MetadataVersion(2, 0).equals(new MetadataVersion(parquetTableMetada * * @param parquetTableMetadata table metadata that should be corrected */ - public static void correctBinaryInMetadataCache(Metadata.ParquetTableMetadataBase parquetTableMetadata) { + public static void correctBinaryInMetadataCache(ParquetTableMetadataBase parquetTableMetadata) { // Looking for the names of the columns with BINARY data type // in the metadata cache file for V2 and all v3 versions Set> columnsNames = getBinaryColumnsNames(parquetTableMetadata); - for (Metadata.ParquetFileMetadata file : parquetTableMetadata.getFiles()) { - for (Metadata.RowGroupMetadata rowGroupMetadata : file.getRowGroups()) { + for (ParquetFileMetadata file : parquetTableMetadata.getFiles()) { + for (RowGroupMetadata rowGroupMetadata : file.getRowGroups()) { Long rowCount = rowGroupMetadata.getRowCount(); - for (Metadata.ColumnMetadata columnMetadata : rowGroupMetadata.getColumns()) { + for (ColumnMetadata columnMetadata : rowGroupMetadata.getColumns()) { // Setting Min/Max values for ParquetTableMetadata_v1 if (new MetadataVersion(1, 0).equals(new MetadataVersion(parquetTableMetadata.getMetadataVersion()))) { if (columnMetadata.getPrimitiveType() == PrimitiveTypeName.BINARY @@ -231,19 +243,19 @@ else if (new MetadataVersion(parquetTableMetadata.getMetadataVersion()).compareT * @param parquetTableMetadata table metadata the source of the columns to check * @return set of the lists with column names */ - private static Set> getBinaryColumnsNames(Metadata.ParquetTableMetadataBase parquetTableMetadata) { + private static Set> getBinaryColumnsNames(ParquetTableMetadataBase parquetTableMetadata) { Set> names = Sets.newHashSet(); - if (parquetTableMetadata instanceof Metadata.ParquetTableMetadata_v2) { - for (Metadata.ColumnTypeMetadata_v2 columnTypeMetadata : - ((Metadata.ParquetTableMetadata_v2) parquetTableMetadata).columnTypeInfo.values()) { + if (parquetTableMetadata instanceof ParquetTableMetadata_v2) { + for (ColumnTypeMetadata_v2 columnTypeMetadata : + ((ParquetTableMetadata_v2) parquetTableMetadata).columnTypeInfo.values()) { if (columnTypeMetadata.primitiveType == PrimitiveTypeName.BINARY || columnTypeMetadata.primitiveType == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { names.add(Arrays.asList(columnTypeMetadata.name)); } } - } else if (parquetTableMetadata instanceof Metadata.ParquetTableMetadata_v3) { - for (Metadata.ColumnTypeMetadata_v3 columnTypeMetadata : - ((Metadata.ParquetTableMetadata_v3) parquetTableMetadata).columnTypeInfo.values()) { + } else if (parquetTableMetadata instanceof ParquetTableMetadata_v3) { + for (ColumnTypeMetadata_v3 columnTypeMetadata : + ((ParquetTableMetadata_v3) parquetTableMetadata).columnTypeInfo.values()) { if (columnTypeMetadata.primitiveType == PrimitiveTypeName.BINARY || columnTypeMetadata.primitiveType == PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { names.add(Arrays.asList(columnTypeMetadata.name)); @@ -260,7 +272,7 @@ private static Set> getBinaryColumnsNames(Metadata.ParquetTableMeta * @param columnMetadata column metadata that should be changed * @param rowCount rows count in column chunk */ - private static void setMinMaxValues(Metadata.ColumnMetadata columnMetadata, long rowCount) { + private static void setMinMaxValues(ColumnMetadata columnMetadata, long rowCount) { if (columnMetadata.hasSingleValue(rowCount)) { Object minValue = columnMetadata.getMinValue(); if (minValue != null && minValue instanceof String) { @@ -278,7 +290,7 @@ private static void setMinMaxValues(Metadata.ColumnMetadata columnMetadata, long * @param columnMetadata column metadata that should be changed * @param rowCount rows count in column chunk */ - private static void convertMinMaxValues(Metadata.ColumnMetadata columnMetadata, long rowCount) { + private static void convertMinMaxValues(ColumnMetadata columnMetadata, long rowCount) { if (columnMetadata.hasSingleValue(rowCount)) { Object minValue = columnMetadata.getMinValue(); if (minValue != null && minValue instanceof String) { @@ -362,8 +374,8 @@ else if (Boolean.valueOf(isDateCorrect)) { * This method only checks the first Row Group, because Drill has only ever written * a single Row Group per file. * - * @param footer - * @param columns + * @param footer parquet footer + * @param columns list of columns schema path * @param autoCorrectCorruptDates user setting to allow enabling/disabling of auto-correction * of corrupt dates. There are some rare cases (storing dates thousands * of years into the future, with tools other than Drill writing files) @@ -459,4 +471,66 @@ public static long getDateTimeValueFromBinary(Binary binaryTimeStampValue, boole } } } + + /** + * Builds major type using given {@code OriginalType originalType} or {@code PrimitiveTypeName type}. + * For DECIMAL will be returned major type with scale and precision. + * + * @param type parquet primitive type + * @param originalType parquet original type + * @param scale type scale (used for DECIMAL type) + * @param precision type precision (used for DECIMAL type) + * @return major type + */ + public static TypeProtos.MajorType getType(PrimitiveTypeName type, OriginalType originalType, int scale, int precision) { + if (originalType != null) { + switch (originalType) { + case DECIMAL: + return Types.withScaleAndPrecision(TypeProtos.MinorType.DECIMAL18, TypeProtos.DataMode.OPTIONAL, scale, precision); + case DATE: + return Types.optional(TypeProtos.MinorType.DATE); + case TIME_MILLIS: + return Types.optional(TypeProtos.MinorType.TIME); + case TIMESTAMP_MILLIS: + return Types.optional(TypeProtos.MinorType.TIMESTAMP); + case UTF8: + return Types.optional(TypeProtos.MinorType.VARCHAR); + case UINT_8: + return Types.optional(TypeProtos.MinorType.UINT1); + case UINT_16: + return Types.optional(TypeProtos.MinorType.UINT2); + case UINT_32: + return Types.optional(TypeProtos.MinorType.UINT4); + case UINT_64: + return Types.optional(TypeProtos.MinorType.UINT8); + case INT_8: + return Types.optional(TypeProtos.MinorType.TINYINT); + case INT_16: + return Types.optional(TypeProtos.MinorType.SMALLINT); + case INTERVAL: + return Types.optional(TypeProtos.MinorType.INTERVAL); + } + } + + switch (type) { + case BOOLEAN: + return Types.optional(TypeProtos.MinorType.BIT); + case INT32: + return Types.optional(TypeProtos.MinorType.INT); + case INT64: + return Types.optional(TypeProtos.MinorType.BIGINT); + case FLOAT: + return Types.optional(TypeProtos.MinorType.FLOAT4); + case DOUBLE: + return Types.optional(TypeProtos.MinorType.FLOAT8); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + case INT96: + return Types.optional(TypeProtos.MinorType.VARBINARY); + default: + // Should never hit this + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } + } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java index f1fb1e9207f..3ccdad0495e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java @@ -17,7 +17,6 @@ */ package org.apache.drill.exec.store.parquet; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; @@ -26,12 +25,9 @@ import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.logical.FormatPluginConfig; import org.apache.drill.common.logical.StoragePluginConfig; -import org.apache.drill.exec.physical.base.AbstractBase; -import org.apache.drill.exec.physical.base.GroupScan; import org.apache.drill.exec.physical.base.PhysicalOperator; -import org.apache.drill.exec.physical.base.PhysicalVisitor; -import org.apache.drill.exec.physical.base.SubScan; import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType; +import org.apache.drill.exec.store.ColumnExplorer; import org.apache.drill.exec.store.StoragePluginRegistry; import com.fasterxml.jackson.annotation.JacksonInject; @@ -40,31 +36,26 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterators; +import org.apache.hadoop.conf.Configuration; -// Class containing information for reading a single parquet row group form HDFS +// Class containing information for reading a single parquet row group from HDFS @JsonTypeName("parquet-row-group-scan") -public class ParquetRowGroupScan extends AbstractBase implements SubScan { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRowGroupScan.class); +public class ParquetRowGroupScan extends AbstractParquetRowGroupScan { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRowGroupScan.class); - private final ParquetFormatConfig formatConfig; private final ParquetFormatPlugin formatPlugin; - private final List rowGroupReadEntries; - private final List columns; - private LogicalExpression filter; - private String selectionRoot; + private final ParquetFormatConfig formatConfig; + private final String selectionRoot; @JsonCreator - public ParquetRowGroupScan( // - @JacksonInject StoragePluginRegistry registry, // - @JsonProperty("userName") String userName, // - @JsonProperty("storage") StoragePluginConfig storageConfig, // - @JsonProperty("format") FormatPluginConfig formatConfig, // - @JsonProperty("entries") LinkedList rowGroupReadEntries, // - @JsonProperty("columns") List columns, // - @JsonProperty("selectionRoot") String selectionRoot, // - @JsonProperty("filter") LogicalExpression filter - ) throws ExecutionSetupException { + public ParquetRowGroupScan(@JacksonInject StoragePluginRegistry registry, + @JsonProperty("userName") String userName, + @JsonProperty("storageConfig") StoragePluginConfig storageConfig, + @JsonProperty("formatConfig") FormatPluginConfig formatConfig, + @JsonProperty("rowGroupReadEntries") LinkedList rowGroupReadEntries, + @JsonProperty("columns") List columns, + @JsonProperty("selectionRoot") String selectionRoot, + @JsonProperty("filter") LogicalExpression filter) throws ExecutionSetupException { this(userName, (ParquetFormatPlugin) registry.getFormatPlugin(Preconditions.checkNotNull(storageConfig), Preconditions.checkNotNull(formatConfig)), rowGroupReadEntries, @@ -73,82 +64,72 @@ public ParquetRowGroupScan( // filter); } - public ParquetRowGroupScan( // - String userName, // - ParquetFormatPlugin formatPlugin, // - List rowGroupReadEntries, // - List columns, // - String selectionRoot, // - LogicalExpression filter - ) { - super(userName); + public ParquetRowGroupScan(String userName, + ParquetFormatPlugin formatPlugin, + List rowGroupReadEntries, + List columns, + String selectionRoot, + LogicalExpression filter) { + super(userName, rowGroupReadEntries, columns, filter); this.formatPlugin = Preconditions.checkNotNull(formatPlugin, "Could not find format config for the given configuration"); this.formatConfig = formatPlugin.getConfig(); - this.rowGroupReadEntries = rowGroupReadEntries; - this.columns = columns == null ? GroupScan.ALL_COLUMNS : columns; this.selectionRoot = selectionRoot; - this.filter = filter; - } - - @JsonProperty("entries") - public List getRowGroupReadEntries() { - return rowGroupReadEntries; } - @JsonProperty("storage") - public StoragePluginConfig getEngineConfig() { + @JsonProperty + public StoragePluginConfig getStorageConfig() { return formatPlugin.getStorageConfig(); } - /** - * @return Parquet plugin format config - */ - @JsonProperty("format") + @JsonProperty public ParquetFormatConfig getFormatConfig() { return formatConfig; } + @JsonProperty public String getSelectionRoot() { return selectionRoot; } - @Override - public boolean isExecutable() { - return false; - } - @JsonIgnore public ParquetFormatPlugin getStorageEngine() { return formatPlugin; } @Override - public T accept(PhysicalVisitor physicalVisitor, X value) throws E { - return physicalVisitor.visitSubScan(this, value); + public PhysicalOperator getNewWithChildren(List children) { + Preconditions.checkArgument(children.isEmpty()); + return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, selectionRoot, filter); } @Override - public PhysicalOperator getNewWithChildren(List children) throws ExecutionSetupException { - Preconditions.checkArgument(children.isEmpty()); - return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, selectionRoot, filter); + public int getOperatorType() { + return CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE; } @Override - public Iterator iterator() { - return Iterators.emptyIterator(); + public AbstractParquetRowGroupScan copy(List columns) { + return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, selectionRoot, filter); } - public List getColumns() { - return columns; + @Override + public boolean areCorruptDatesAutoCorrected() { + return formatConfig.areCorruptDatesAutoCorrected(); } - public LogicalExpression getFilter() { - return filter; + @Override + public Configuration getFsConf(RowGroupReadEntry rowGroupReadEntry) { + return formatPlugin.getFsConf(); } @Override - public int getOperatorType() { - return CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE; + public boolean supportsFileImplicitColumns() { + return selectionRoot != null; } + @Override + public List getPartitionValues(RowGroupReadEntry rowGroupReadEntry) { + return ColumnExplorer.listPartitionValues(rowGroupReadEntry.getPath(), selectionRoot); + } } + diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java index e928ebbb5e1..c71e3c3c19e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java @@ -17,15 +17,7 @@ */ package org.apache.drill.exec.store.parquet; -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import com.google.common.base.Functions; -import com.google.common.base.Stopwatch; -import com.google.common.collect.Maps; +import com.google.common.base.Preconditions; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.ops.ExecutorFragmentContext; @@ -33,144 +25,52 @@ import org.apache.drill.exec.physical.impl.BatchCreator; import org.apache.drill.exec.physical.impl.ScanBatch; import org.apache.drill.exec.record.RecordBatch; -import org.apache.drill.exec.store.ColumnExplorer; -import org.apache.drill.exec.store.RecordReader; +import org.apache.drill.exec.server.options.OptionManager; import org.apache.drill.exec.store.dfs.DrillFileSystem; -import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader; -import org.apache.drill.exec.store.parquet2.DrillParquetReader; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.hadoop.CodecFactory; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.Type; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; - -public class ParquetScanBatchCreator implements BatchCreator{ - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetScanBatchCreator.class); +import java.io.IOException; +import java.util.List; - private static final String ENABLE_BYTES_READ_COUNTER = "parquet.benchmark.bytes.read"; - private static final String ENABLE_BYTES_TOTAL_COUNTER = "parquet.benchmark.bytes.total"; - private static final String ENABLE_TIME_READ_COUNTER = "parquet.benchmark.time.read"; +public class ParquetScanBatchCreator extends AbstractParquetScanBatchCreator implements BatchCreator { - @SuppressWarnings("resource") @Override - public ScanBatch getBatch(ExecutorFragmentContext context, ParquetRowGroupScan rowGroupScan, List children) - throws ExecutionSetupException { + public ScanBatch getBatch(ExecutorFragmentContext context, ParquetRowGroupScan rowGroupScan, List children) throws ExecutionSetupException { Preconditions.checkArgument(children.isEmpty()); OperatorContext oContext = context.newOperatorContext(rowGroupScan); + return getBatch(context, rowGroupScan, oContext); + } - final ColumnExplorer columnExplorer = new ColumnExplorer(context.getOptions(), rowGroupScan.getColumns()); - - if (!columnExplorer.isStarQuery()) { - rowGroupScan = new ParquetRowGroupScan(rowGroupScan.getUserName(), rowGroupScan.getStorageEngine(), - rowGroupScan.getRowGroupReadEntries(), columnExplorer.getTableColumns(), rowGroupScan.getSelectionRoot(), - rowGroupScan.getFilter()); - rowGroupScan.setOperatorId(rowGroupScan.getOperatorId()); - } - - DrillFileSystem fs; - try { - boolean useAsyncPageReader = - context.getOptions().getOption(ExecConstants.PARQUET_PAGEREADER_ASYNC).bool_val; - if (useAsyncPageReader) { - fs = oContext.newNonTrackingFileSystem(rowGroupScan.getStorageEngine().getFsConf()); - } else { - fs = oContext.newFileSystem(rowGroupScan.getStorageEngine().getFsConf()); - } - } catch (IOException e) { - throw new ExecutionSetupException( - String.format("Failed to create DrillFileSystem: %s", e.getMessage()), e); - } - - Configuration conf = new Configuration(fs.getConf()); - conf.setBoolean(ENABLE_BYTES_READ_COUNTER, false); - conf.setBoolean(ENABLE_BYTES_TOTAL_COUNTER, false); - conf.setBoolean(ENABLE_TIME_READ_COUNTER, false); + @Override + protected AbstractDrillFileSystemManager getDrillFileSystemCreator(OperatorContext operatorContext, OptionManager optionManager) { + return new ParquetDrillFileSystemManager(operatorContext, optionManager.getOption(ExecConstants.PARQUET_PAGEREADER_ASYNC).bool_val); + } - // keep footers in a map to avoid re-reading them - Map footers = Maps.newHashMap(); - List readers = new LinkedList<>(); - List> implicitColumns = Lists.newArrayList(); - Map mapWithMaxColumns = Maps.newLinkedHashMap(); - for(RowGroupReadEntry e : rowGroupScan.getRowGroupReadEntries()){ - /* - Here we could store a map from file names to footers, to prevent re-reading the footer for each row group in a file - TODO - to prevent reading the footer again in the parquet record reader (it is read earlier in the ParquetStorageEngine) - we should add more information to the RowGroupInfo that will be populated upon the first read to - provide the reader with all of th file meta-data it needs - These fields will be added to the constructor below - */ - try { - Stopwatch timer = Stopwatch.createUnstarted(); - if (!footers.containsKey(e.getPath())){ - timer.start(); - ParquetMetadata footer = ParquetFileReader.readFooter(conf, new Path(e.getPath())); - long timeToRead = timer.elapsed(TimeUnit.MICROSECONDS); - logger.trace("ParquetTrace,Read Footer,{},{},{},{},{},{},{}", "", e.getPath(), "", 0, 0, 0, timeToRead); - footers.put(e.getPath(), footer ); - } - boolean autoCorrectCorruptDates = rowGroupScan.getFormatConfig().areCorruptDatesAutoCorrected(); - ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = ParquetReaderUtility.detectCorruptDates(footers.get(e.getPath()), rowGroupScan.getColumns(), - autoCorrectCorruptDates); - if (logger.isDebugEnabled()) { - logger.debug(containsCorruptDates.toString()); - } - if (!context.getOptions().getBoolean(ExecConstants.PARQUET_NEW_RECORD_READER) && !isComplex(footers.get(e.getPath()))) { - readers.add( - new ParquetRecordReader( - context, e.getPath(), e.getRowGroupIndex(), e.getNumRecordsToRead(), fs, - CodecFactory.createDirectCodecFactory( - fs.getConf(), - new ParquetDirectByteBufferAllocator(oContext.getAllocator()), 0), - footers.get(e.getPath()), - rowGroupScan.getColumns(), - containsCorruptDates - ) - ); - } else { - ParquetMetadata footer = footers.get(e.getPath()); - readers.add(new DrillParquetReader(context, footer, e, columnExplorer.getTableColumns(), fs, containsCorruptDates)); - } - Map implicitValues = columnExplorer.populateImplicitColumns(e, rowGroupScan.getSelectionRoot()); - implicitColumns.add(implicitValues); - if (implicitValues.size() > mapWithMaxColumns.size()) { - mapWithMaxColumns = implicitValues; - } + /** + * Creates file system only if it was not created before, otherwise returns already created instance. + */ + private class ParquetDrillFileSystemManager extends AbstractDrillFileSystemManager { - } catch (IOException e1) { - throw new ExecutionSetupException(e1); - } - } + private final boolean useAsyncPageReader; + private DrillFileSystem fs; - // all readers should have the same number of implicit columns, add missing ones with value null - Map diff = Maps.transformValues(mapWithMaxColumns, Functions.constant((String) null)); - for (Map map : implicitColumns) { - map.putAll(Maps.difference(map, diff).entriesOnlyOnRight()); + ParquetDrillFileSystemManager(OperatorContext operatorContext, boolean useAsyncPageReader) { + super(operatorContext); + this.useAsyncPageReader = useAsyncPageReader; } - return new ScanBatch(context, oContext, readers, implicitColumns); - } - - private static boolean isComplex(ParquetMetadata footer) { - MessageType schema = footer.getFileMetaData().getSchema(); - - for (Type type : schema.getFields()) { - if (!type.isPrimitive()) { - return true; - } - } - for (ColumnDescriptor col : schema.getColumns()) { - if (col.getMaxRepetitionLevel() > 0) { - return true; + @Override + protected DrillFileSystem get(Configuration config, String path) throws ExecutionSetupException { + if (fs == null) { + try { + fs = useAsyncPageReader ? operatorContext.newNonTrackingFileSystem(config) : operatorContext.newFileSystem(config); + } catch (IOException e) { + throw new ExecutionSetupException(String.format("Failed to create DrillFileSystem: %s", e.getMessage()), e); + } } + return fs; } - return false; } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java new file mode 100644 index 00000000000..af436d8fc93 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.drill.exec.store.dfs.ReadEntryFromHDFS; +import org.apache.drill.exec.store.dfs.easy.FileWork; +import org.apache.drill.exec.store.schedule.CompleteWork; +import org.apache.drill.exec.store.schedule.EndpointByteMap; + +import java.util.List; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; + +public class RowGroupInfo extends ReadEntryFromHDFS implements CompleteWork, FileWork { + + private EndpointByteMap byteMap; + private int rowGroupIndex; + private List columns; + private long rowCount; // rowCount = -1 indicates to include all rows. + private long numRecordsToRead; + + @JsonCreator + public RowGroupInfo(@JsonProperty("path") String path, + @JsonProperty("start") long start, + @JsonProperty("length") long length, + @JsonProperty("rowGroupIndex") int rowGroupIndex, + long rowCount) { + super(path, start, length); + this.rowGroupIndex = rowGroupIndex; + this.rowCount = rowCount; + this.numRecordsToRead = rowCount; + } + + public RowGroupReadEntry getRowGroupReadEntry() { + return new RowGroupReadEntry(this.getPath(), this.getStart(), this.getLength(), this.rowGroupIndex, this.getNumRecordsToRead()); + } + + public int getRowGroupIndex() { + return this.rowGroupIndex; + } + + @Override + public int compareTo(CompleteWork o) { + return Long.compare(getTotalBytes(), o.getTotalBytes()); + } + + @Override + public long getTotalBytes() { + return this.getLength(); + } + + @Override + public EndpointByteMap getByteMap() { + return byteMap; + } + + public long getNumRecordsToRead() { + return numRecordsToRead; + } + + public void setNumRecordsToRead(long numRecords) { + numRecordsToRead = numRecords; + } + + public void setEndpointByteMap(EndpointByteMap byteMap) { + this.byteMap = byteMap; + } + + public long getRowCount() { + return rowCount; + } + + public List getColumns() { + return columns; + } + + public void setColumns(List columns) { + this.columns = columns; + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java new file mode 100644 index 00000000000..cdb28c2d4da --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java @@ -0,0 +1,694 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator.Feature; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; +import com.google.common.base.Stopwatch; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.drill.common.util.DrillVersionInfo; +import org.apache.drill.exec.store.TimedRunnable; +import org.apache.drill.exec.store.dfs.MetadataContext; +import org.apache.drill.exec.store.parquet.ParquetFormatConfig; +import org.apache.drill.exec.store.parquet.ParquetReaderUtility; +import org.apache.drill.exec.util.DrillFileSystemUtil; +import org.apache.drill.exec.util.ImpersonationUtil; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.RowGroupMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.SUPPORTED_VERSIONS; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ColumnMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ColumnTypeMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetFileMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTableMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.RowGroupMetadata_v3; + +public class Metadata { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class); + + 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 ParquetFormatConfig formatConfig; + + private ParquetTableMetadataBase parquetTableMetadata; + private ParquetTableMetadataDirs parquetTableMetadataDirs; + + + private Metadata(ParquetFormatConfig formatConfig) { + this.formatConfig = formatConfig; + } + + /** + * Create the parquet metadata file for the directory at the given path, and for any subdirectories. + * + * @param fs file system + * @param path path + */ + public static void createMeta(FileSystem fs, String path, ParquetFormatConfig formatConfig) throws IOException { + Metadata metadata = new Metadata(formatConfig); + metadata.createMetaFilesRecursively(path, fs); + } + + /** + * Get the parquet metadata for the parquet files in the given directory, including those in subdirectories. + * + * @param fs file system + * @param path path + * @return parquet table metadata + */ + public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs, String path, ParquetFormatConfig formatConfig) + throws IOException { + Metadata metadata = new Metadata(formatConfig); + return metadata.getParquetTableMetadata(path, fs); + } + + /** + * Get the parquet metadata for a list of parquet files. + * + * @param fileStatusMap file statuses and corresponding file systems + * @param formatConfig parquet format config + * @return parquet table metadata + */ + public static ParquetTableMetadata_v3 getParquetTableMetadata(Map fileStatusMap, + ParquetFormatConfig formatConfig) throws IOException { + Metadata metadata = new Metadata(formatConfig); + return metadata.getParquetTableMetadata(fileStatusMap); + } + + /** + * Get the parquet metadata for the table by reading the metadata file + * + * @param fs current file system + * @param path The path to the metadata file, located in the directory that contains the parquet files + * @param metaContext metadata context + * @param formatConfig parquet format plugin configs + * @return parquet table metadata. Null if metadata cache is missing, unsupported or corrupted + */ + public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext, + ParquetFormatConfig formatConfig) { + if (ignoreReadingMetadata(metaContext, path)) { + return null; + } + Metadata metadata = new Metadata(formatConfig); + metadata.readBlockMeta(path, false, metaContext, fs); + return metadata.parquetTableMetadata; + } + + /** + * Get the parquet metadata for all subdirectories by reading the metadata file + * + * @param fs current file system + * @param path The path to the metadata file, located in the directory that contains the parquet files + * @param metaContext metadata context + * @param formatConfig parquet format plugin configs + * @return parquet metadata for a directory. Null if metadata cache is missing, unsupported or corrupted + */ + public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path, + MetadataContext metaContext, ParquetFormatConfig formatConfig) { + if (ignoreReadingMetadata(metaContext, path)) { + return null; + } + Metadata metadata = new Metadata(formatConfig); + metadata.readBlockMeta(path, true, metaContext, fs); + return metadata.parquetTableMetadataDirs; + } + + /** + * Ignore reading metadata files, if metadata is missing, unsupported or corrupted + * + * @param metaContext Metadata context + * @param path The path to the metadata file, located in the directory that contains the parquet files + * @return true if parquet metadata is missing or corrupted, false otherwise + */ + private static boolean ignoreReadingMetadata(MetadataContext metaContext, Path path) { + if (metaContext.isMetadataCacheCorrupted()) { + logger.warn("Ignoring of reading '{}' metadata file. Parquet metadata cache files are unsupported or corrupted. " + + "Query performance may be slow. Make sure the cache files are up-to-date by running the 'REFRESH TABLE " + + "METADATA' command", path); + return true; + } + return false; + } + + /** + * Create the parquet metadata files for the directory at the given path and for any subdirectories. + * Metadata cache files written to the disk contain relative paths. Returned Pair of metadata contains absolute paths. + * + * @param path to the directory of the parquet table + * @param fs file system + * @return Pair of parquet metadata. The left one is a parquet metadata for the table. The right one of the Pair is + * a metadata for all subdirectories (if they are present and there are no any parquet files in the + * {@code path} directory). + * @throws IOException if parquet metadata can't be serialized and written to the json file + */ + private Pair createMetaFilesRecursively(final String path, FileSystem fs) throws IOException { + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; + List metaDataList = Lists.newArrayList(); + List directoryList = Lists.newArrayList(); + ConcurrentHashMap columnTypeInfoSet = + new ConcurrentHashMap<>(); + Path p = new Path(path); + FileStatus fileStatus = fs.getFileStatus(p); + assert fileStatus.isDirectory() : "Expected directory"; + + final Map childFiles = new LinkedHashMap<>(); + + for (final FileStatus file : DrillFileSystemUtil.listAll(fs, p, false)) { + if (file.isDirectory()) { + ParquetTableMetadata_v3 subTableMetadata = (createMetaFilesRecursively(file.getPath().toString(), fs)).getLeft(); + metaDataList.addAll(subTableMetadata.files); + directoryList.addAll(subTableMetadata.directories); + directoryList.add(file.getPath().toString()); + // Merge the schema from the child level into the current level + //TODO: We need a merge method that merges two columns with the same name but different types + columnTypeInfoSet.putAll(subTableMetadata.columnTypeInfo); + } else { + childFiles.put(file, fs); + } + } + ParquetTableMetadata_v3 parquetTableMetadata = new ParquetTableMetadata_v3(SUPPORTED_VERSIONS.last().toString(), + DrillVersionInfo.getVersion()); + if (childFiles.size() > 0) { + List childFilesMetadata = getParquetFileMetadata_v3(parquetTableMetadata, childFiles); + metaDataList.addAll(childFilesMetadata); + // Note that we do not need to merge the columnInfo at this point. The columnInfo is already added + // to the parquetTableMetadata. + } + + parquetTableMetadata.directories = directoryList; + parquetTableMetadata.files = metaDataList; + // TODO: We need a merge method that merges two columns with the same name but different types + if (parquetTableMetadata.columnTypeInfo == null) { + parquetTableMetadata.columnTypeInfo = new ConcurrentHashMap<>(); + } + parquetTableMetadata.columnTypeInfo.putAll(columnTypeInfoSet); + + for (String oldName : OLD_METADATA_FILENAMES) { + fs.delete(new Path(p, oldName), false); + } + // relative paths in the metadata are only necessary for meta cache files. + ParquetTableMetadata_v3 metadataTableWithRelativePaths = + MetadataPathUtils.createMetadataWithRelativePaths(parquetTableMetadata, path); + writeFile(metadataTableWithRelativePaths, new Path(p, METADATA_FILENAME), fs); + + if (directoryList.size() > 0 && childFiles.size() == 0) { + ParquetTableMetadataDirs parquetTableMetadataDirsRelativePaths = + new ParquetTableMetadataDirs(metadataTableWithRelativePaths.directories); + writeFile(parquetTableMetadataDirsRelativePaths, new Path(p, METADATA_DIRECTORIES_FILENAME), fs); + if (timer != null) { + logger.debug("Creating metadata files recursively took {} ms", timer.elapsed(TimeUnit.MILLISECONDS)); + } + ParquetTableMetadataDirs parquetTableMetadataDirs = new ParquetTableMetadataDirs(directoryList); + return Pair.of(parquetTableMetadata, parquetTableMetadataDirs); + } + List emptyDirList = Lists.newArrayList(); + if (timer != null) { + logger.debug("Creating metadata files recursively took {} ms", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + } + return Pair.of(parquetTableMetadata, new ParquetTableMetadataDirs(emptyDirList)); + } + + /** + * Get the parquet metadata for the parquet files in a directory. + * + * @param path the path of the directory + * @return metadata object for an entire parquet directory structure + * @throws IOException in case of problems during accessing files + */ + private ParquetTableMetadata_v3 getParquetTableMetadata(String path, FileSystem fs) throws IOException { + Path p = new Path(path); + FileStatus fileStatus = fs.getFileStatus(p); + Stopwatch watch = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; + List fileStatuses = new ArrayList<>(); + if (fileStatus.isFile()) { + fileStatuses.add(fileStatus); + } else { + fileStatuses.addAll(DrillFileSystemUtil.listFiles(fs, p, true)); + } + if (watch != null) { + logger.debug("Took {} ms to get file statuses", watch.elapsed(TimeUnit.MILLISECONDS)); + watch.reset(); + watch.start(); + } + + Map fileStatusMap = fileStatuses.stream() + .collect( + Collectors.toMap( + Function.identity(), + s -> fs, + (oldFs, newFs) -> newFs, + LinkedHashMap::new)); + + ParquetTableMetadata_v3 metadata_v3 = getParquetTableMetadata(fileStatusMap); + if (watch != null) { + logger.debug("Took {} ms to read file metadata", watch.elapsed(TimeUnit.MILLISECONDS)); + watch.stop(); + } + return metadata_v3; + } + + /** + * Get the parquet metadata for a list of parquet files + * + * @param fileStatusMap file statuses and corresponding file systems + * @return parquet table metadata object + * @throws IOException if parquet file metadata can't be obtained + */ + private ParquetTableMetadata_v3 getParquetTableMetadata(Map fileStatusMap) + throws IOException { + ParquetTableMetadata_v3 tableMetadata = new ParquetTableMetadata_v3(SUPPORTED_VERSIONS.last().toString(), + DrillVersionInfo.getVersion()); + tableMetadata.files = getParquetFileMetadata_v3(tableMetadata, fileStatusMap); + tableMetadata.directories = new ArrayList<>(); + return tableMetadata; + } + + /** + * Get a list of file metadata for a list of parquet files + * + * @param parquetTableMetadata_v3 can store column schema info from all the files and row groups + * @param fileStatusMap parquet files statuses and corresponding file systems + * + * @return list of the parquet file metadata with absolute paths + * @throws IOException is thrown in case of issues while executing the list of runnables + */ + private List getParquetFileMetadata_v3( + ParquetTableMetadata_v3 parquetTableMetadata_v3, Map fileStatusMap) throws IOException { + + List> gatherers = fileStatusMap.entrySet().stream() + .map(e -> new MetadataGatherer(parquetTableMetadata_v3, e.getKey(), e.getValue())) + .collect(Collectors.toList()); + + List metaDataList = new ArrayList<>(); + metaDataList.addAll(TimedRunnable.run("Fetch parquet metadata", logger, gatherers, 16)); + return metaDataList; + } + + /** + * TimedRunnable that reads the footer from parquet and collects file metadata + */ + private class MetadataGatherer extends TimedRunnable { + + private final ParquetTableMetadata_v3 parquetTableMetadata; + private final FileStatus fileStatus; + private final FileSystem fs; + + MetadataGatherer(ParquetTableMetadata_v3 parquetTableMetadata, FileStatus fileStatus, FileSystem fs) { + this.parquetTableMetadata = parquetTableMetadata; + this.fileStatus = fileStatus; + this.fs = fs; + } + + @Override + protected ParquetFileMetadata_v3 runInner() throws Exception { + return getParquetFileMetadata_v3(parquetTableMetadata, fileStatus, fs); + } + + @Override + protected IOException convertToIOException(Exception e) { + if (e instanceof IOException) { + return (IOException) e; + } else { + return new IOException(e); + } + } + } + + private ColTypeInfo getColTypeInfo(MessageType schema, Type type, String[] path, int depth) { + if (type.isPrimitive()) { + PrimitiveType primitiveType = (PrimitiveType) type; + int precision = 0; + int scale = 0; + if (primitiveType.getDecimalMetadata() != null) { + precision = primitiveType.getDecimalMetadata().getPrecision(); + scale = primitiveType.getDecimalMetadata().getScale(); + } + + int repetitionLevel = schema.getMaxRepetitionLevel(path); + int definitionLevel = schema.getMaxDefinitionLevel(path); + + return new ColTypeInfo(type.getOriginalType(), precision, scale, repetitionLevel, definitionLevel); + } + Type t = ((GroupType) type).getType(path[depth]); + return getColTypeInfo(schema, t, path, depth + 1); + } + + private class ColTypeInfo { + public OriginalType originalType; + public int precision; + public int scale; + public int repetitionLevel; + public int definitionLevel; + + ColTypeInfo(OriginalType originalType, int precision, int scale, int repetitionLevel, int definitionLevel) { + this.originalType = originalType; + this.precision = precision; + this.scale = scale; + this.repetitionLevel = repetitionLevel; + this.definitionLevel = definitionLevel; + } + } + + /** + * Get the metadata for a single file + */ + private ParquetFileMetadata_v3 getParquetFileMetadata_v3(ParquetTableMetadata_v3 parquetTableMetadata, + final FileStatus file, final FileSystem fs) throws IOException, InterruptedException { + final ParquetMetadata metadata; + final UserGroupInformation processUserUgi = ImpersonationUtil.getProcessUserUGI(); + try { + metadata = processUserUgi.doAs((PrivilegedExceptionAction) + () -> ParquetFileReader.readFooter(fs.getConf(), file, ParquetMetadataConverter.NO_FILTER)); + } catch(Exception e) { + logger.error("Exception while reading footer of parquet file [Details - path: {}, owner: {}] as process user {}", + file.getPath(), file.getOwner(), processUserUgi.getShortUserName(), e); + throw e; + } + + MessageType schema = metadata.getFileMetaData().getSchema(); + +// Map originalTypeMap = Maps.newHashMap(); + Map colTypeInfoMap = Maps.newHashMap(); + schema.getPaths(); + for (String[] path : schema.getPaths()) { + colTypeInfoMap.put(SchemaPath.getCompoundPath(path), getColTypeInfo(schema, schema, path, 0)); + } + + List rowGroupMetadataList = Lists.newArrayList(); + + ArrayList ALL_COLS = new ArrayList<>(); + ALL_COLS.add(SchemaPath.STAR_COLUMN); + boolean autoCorrectCorruptDates = formatConfig.areCorruptDatesAutoCorrected(); + ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = ParquetReaderUtility.detectCorruptDates(metadata, ALL_COLS, autoCorrectCorruptDates); + if (logger.isDebugEnabled()) { + logger.debug(containsCorruptDates.toString()); + } + for (BlockMetaData rowGroup : metadata.getBlocks()) { + List columnMetadataList = Lists.newArrayList(); + long length = 0; + for (ColumnChunkMetaData col : rowGroup.getColumns()) { + ColumnMetadata_v3 columnMetadata; + + boolean statsAvailable = (col.getStatistics() != null && !col.getStatistics().isEmpty()); + + Statistics stats = col.getStatistics(); + String[] columnName = col.getPath().toArray(); + SchemaPath columnSchemaName = SchemaPath.getCompoundPath(columnName); + ColTypeInfo colTypeInfo = colTypeInfoMap.get(columnSchemaName); + + ColumnTypeMetadata_v3 columnTypeMetadata = + new ColumnTypeMetadata_v3(columnName, col.getType(), colTypeInfo.originalType, + colTypeInfo.precision, colTypeInfo.scale, colTypeInfo.repetitionLevel, colTypeInfo.definitionLevel); + + if (parquetTableMetadata.columnTypeInfo == null) { + parquetTableMetadata.columnTypeInfo = new ConcurrentHashMap<>(); + } + // Save the column schema info. We'll merge it into one list + parquetTableMetadata.columnTypeInfo + .put(new ColumnTypeMetadata_v3.Key(columnTypeMetadata.name), columnTypeMetadata); + if (statsAvailable) { + // Write stats when they are not null + Object minValue = null; + Object maxValue = null; + if (stats.genericGetMax() != null && stats.genericGetMin() != null ) { + minValue = stats.genericGetMin(); + maxValue = stats.genericGetMax(); + if (containsCorruptDates == ParquetReaderUtility.DateCorruptionStatus.META_SHOWS_CORRUPTION + && columnTypeMetadata.originalType == OriginalType.DATE) { + minValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) minValue); + maxValue = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) maxValue); + } + + } + columnMetadata = + new ColumnMetadata_v3(columnTypeMetadata.name, col.getType(), minValue, maxValue, stats.getNumNulls()); + } else { + columnMetadata = new ColumnMetadata_v3(columnTypeMetadata.name, col.getType(), null, null, null); + } + columnMetadataList.add(columnMetadata); + length += col.getTotalSize(); + } + + // DRILL-5009: Skip the RowGroup if it is empty + // Note we still read the schema even if there are no values in the RowGroup + if (rowGroup.getRowCount() == 0) { + continue; + } + RowGroupMetadata_v3 rowGroupMeta = + new RowGroupMetadata_v3(rowGroup.getStartingPos(), length, rowGroup.getRowCount(), + getHostAffinity(file, fs, rowGroup.getStartingPos(), length), columnMetadataList); + + rowGroupMetadataList.add(rowGroupMeta); + } + String path = Path.getPathWithoutSchemeAndAuthority(file.getPath()).toString(); + + return new ParquetFileMetadata_v3(path, file.getLen(), rowGroupMetadataList); + } + + /** + * Get the host affinity for a row group. + * + * @param fileStatus the parquet file + * @param start the start of the row group + * @param length the length of the row group + * @return host affinity for the row group + */ + private Map getHostAffinity(FileStatus fileStatus, FileSystem fs, long start, long length) + throws IOException { + BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, start, length); + Map hostAffinityMap = Maps.newHashMap(); + for (BlockLocation blockLocation : blockLocations) { + for (String host : blockLocation.getHosts()) { + Float currentAffinity = hostAffinityMap.get(host); + float blockStart = blockLocation.getOffset(); + float blockEnd = blockStart + blockLocation.getLength(); + float rowGroupEnd = start + length; + Float newAffinity = (blockLocation.getLength() - (blockStart < start ? start - blockStart : 0) - + (blockEnd > rowGroupEnd ? blockEnd - rowGroupEnd : 0)) / length; + if (currentAffinity != null) { + hostAffinityMap.put(host, currentAffinity + newAffinity); + } else { + hostAffinityMap.put(host, newAffinity); + } + } + } + return hostAffinityMap; + } + + /** + * Serialize parquet metadata to json and write to a file. + * + * @param parquetTableMetadata parquet table metadata + * @param p file path + */ + private void writeFile(ParquetTableMetadata_v3 parquetTableMetadata, Path p, FileSystem fs) 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(); + module.addSerializer(ColumnMetadata_v3.class, new ColumnMetadata_v3.Serializer()); + mapper.registerModule(module); + FSDataOutputStream os = fs.create(p); + mapper.writerWithDefaultPrettyPrinter().writeValue(os, parquetTableMetadata); + os.flush(); + os.close(); + } + + private void writeFile(ParquetTableMetadataDirs parquetTableMetadataDirs, Path p, FileSystem fs) 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 + * + * @param path to metadata file + * @param dirsOnly true for {@link Metadata#METADATA_DIRECTORIES_FILENAME} + * or false for {@link Metadata#METADATA_FILENAME} files reading + * @param metaContext current metadata context + */ + private void readBlockMeta(Path path, boolean dirsOnly, MetadataContext metaContext, FileSystem fs) { + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; + Path metadataParentDir = Path.getPathWithoutSchemeAndAuthority(path.getParent()); + String metadataParentDirPath = metadataParentDir.toUri().getPath(); + ObjectMapper mapper = new ObjectMapper(); + + final SimpleModule serialModule = new SimpleModule(); + serialModule.addDeserializer(SchemaPath.class, new SchemaPath.De()); + serialModule.addKeyDeserializer(Metadata_V2.ColumnTypeMetadata_v2.Key.class, new Metadata_V2.ColumnTypeMetadata_v2.Key.DeSerializer()); + serialModule.addKeyDeserializer(ColumnTypeMetadata_v3.Key.class, new ColumnTypeMetadata_v3.Key.DeSerializer()); + + AfterburnerModule module = new AfterburnerModule(); + module.setUseOptimizedBeanDeserializer(true); + + mapper.registerModule(serialModule); + mapper.registerModule(module); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + try (FSDataInputStream is = fs.open(path)) { + boolean alreadyCheckedModification; + boolean newMetadata = false; + alreadyCheckedModification = metaContext.getStatus(metadataParentDirPath); + + if (dirsOnly) { + parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class); + if (timer != null) { + logger.debug("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + } + parquetTableMetadataDirs.updateRelativePaths(metadataParentDirPath); + if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), path, metadataParentDir, metaContext, fs)) { + parquetTableMetadataDirs = + (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString(), fs)).getRight(); + newMetadata = true; + } + } else { + parquetTableMetadata = mapper.readValue(is, ParquetTableMetadataBase.class); + if (timer != null) { + logger.debug("Took {} ms to read metadata from cache file", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); + } + if (new MetadataVersion(parquetTableMetadata.getMetadataVersion()).compareTo(new MetadataVersion(3, 0)) >= 0) { + ((ParquetTableMetadata_v3) parquetTableMetadata).updateRelativePaths(metadataParentDirPath); + } + if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), path, metadataParentDir, metaContext, fs)) { + parquetTableMetadata = + (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString(), fs)).getLeft(); + newMetadata = true; + } + + // DRILL-5009: Remove the RowGroup if it is empty + List files = parquetTableMetadata.getFiles(); + for (ParquetFileMetadata file : files) { + List rowGroups = file.getRowGroups(); + for (Iterator iter = rowGroups.iterator(); iter.hasNext(); ) { + RowGroupMetadata r = iter.next(); + if (r.getRowCount() == 0) { + iter.remove(); + } + } + } + + } + if (newMetadata) { + // if new metadata files were created, invalidate the existing metadata context + metaContext.clear(); + } + } catch (IOException e) { + logger.error("Failed to read '{}' metadata file", path, e); + metaContext.setMetadataCacheCorrupted(true); + } + } + + /** + * 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 + * + * @param directories List of directories + * @param metaFilePath path of parquet metadata cache file + * @return true if metadata needs to be updated, false otherwise + * @throws IOException if some resources are not accessible + */ + private boolean tableModified(List directories, Path metaFilePath, Path parentDir, MetadataContext metaContext, FileSystem fs) throws IOException { + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; + metaContext.setStatus(parentDir.toUri().getPath()); + long metaFileModifyTime = fs.getFileStatus(metaFilePath).getModificationTime(); + FileStatus directoryStatus = fs.getFileStatus(parentDir); + int numDirs = 1; + if (directoryStatus.getModificationTime() > metaFileModifyTime) { + if (timer != null) { + logger.debug("Directory {} was modified. Took {} ms to check modification time of {} directories", + directoryStatus.getPath().toString(), timer.elapsed(TimeUnit.MILLISECONDS), numDirs); + timer.stop(); + } + return true; + } + for (String directory : directories) { + numDirs++; + metaContext.setStatus(directory); + directoryStatus = fs.getFileStatus(new Path(directory)); + if (directoryStatus.getModificationTime() > metaFileModifyTime) { + if (timer != null) { + logger.debug("Directory {} was modified. Took {} ms to check modification time of {} directories", + directoryStatus.getPath().toString(), timer.elapsed(TimeUnit.MILLISECONDS), numDirs); + timer.stop(); + } + return true; + } + } + if (timer != null) { + logger.debug("No directories were modified. Took {} ms to check modification time of {} directories", + timer.elapsed(TimeUnit.MILLISECONDS), numDirs); + timer.stop(); + } + return false; + } + +} + diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java new file mode 100644 index 00000000000..d7d56c36edf --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataBase.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.util.List; +import java.util.Map; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V1; +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V2; +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V3; +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V3_1; +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V3_2; +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V3_3; + +public class MetadataBase { + + /** + * Basic class for parquet metadata. Inheritors of this class are json serializable structures which contain + * different metadata versions for an entire parquet directory structure + *

+ * If any new code changes affect on the metadata files content, please update metadata version in such manner: + * Bump up metadata major version if metadata structure is changed. + * Bump up metadata minor version if only metadata content is changed, but metadata structure is the same. + *

+ * Note: keep metadata versions synchronized with {@link MetadataVersion.Constants} + */ + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "metadata_version", + visible = true) + @JsonSubTypes({ + @JsonSubTypes.Type(value = Metadata_V1.ParquetTableMetadata_v1.class, name = V1), + @JsonSubTypes.Type(value = Metadata_V2.ParquetTableMetadata_v2.class, name = V2), + @JsonSubTypes.Type(value = Metadata_V3.ParquetTableMetadata_v3.class, name = V3), + @JsonSubTypes.Type(value = Metadata_V3.ParquetTableMetadata_v3.class, name = V3_1), + @JsonSubTypes.Type(value = Metadata_V3.ParquetTableMetadata_v3.class, name = V3_2), + @JsonSubTypes.Type(value = Metadata_V3.ParquetTableMetadata_v3.class, name = V3_3) + }) + public static abstract class ParquetTableMetadataBase { + + @JsonIgnore + public abstract List getDirectories(); + + @JsonIgnore public abstract List getFiles(); + + @JsonIgnore public abstract void assignFiles(List newFiles); + + public abstract boolean hasColumnMetadata(); + + @JsonIgnore public abstract PrimitiveType.PrimitiveTypeName getPrimitiveType(String[] columnName); + + @JsonIgnore public abstract OriginalType getOriginalType(String[] columnName); + + @JsonIgnore public abstract Integer getRepetitionLevel(String[] columnName); + + @JsonIgnore public abstract Integer getDefinitionLevel(String[] columnName); + + @JsonIgnore public abstract boolean isRowGroupPrunable(); + + @JsonIgnore public abstract ParquetTableMetadataBase clone(); + + @JsonIgnore public abstract String getDrillVersion(); + + @JsonIgnore public abstract String getMetadataVersion(); + } + + public static abstract class ParquetFileMetadata { + @JsonIgnore public abstract String getPath(); + + @JsonIgnore public abstract Long getLength(); + + @JsonIgnore public abstract List getRowGroups(); + } + + + public static abstract class RowGroupMetadata { + @JsonIgnore public abstract Long getStart(); + + @JsonIgnore public abstract Long getLength(); + + @JsonIgnore public abstract Long getRowCount(); + + @JsonIgnore public abstract Map getHostAffinity(); + + @JsonIgnore public abstract List getColumns(); + } + + + public static abstract class ColumnMetadata { + public abstract String[] getName(); + + public abstract Long getNulls(); + + public abstract boolean hasSingleValue(long rowCount); + + public abstract Object getMinValue(); + + public abstract Object getMaxValue(); + + /** + * Set the max value recorded in the parquet metadata statistics. + * + * This object would just be immutable, but due to Drill-4203 we need to correct + * date values that had been corrupted by earlier versions of Drill. + */ + public abstract void setMax(Object newMax); + + /** + * Set the min value recorded in the parquet metadata statistics. + * + * This object would just be immutable, but due to Drill-4203 we need to correct + * date values that had been corrupted by earlier versions of Drill. + */ + public abstract void setMin(Object newMax); + + public abstract PrimitiveType.PrimitiveTypeName getPrimitiveType(); + + public abstract OriginalType getOriginalType(); + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java new file mode 100644 index 00000000000..b9480e836cc --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataPathUtils.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.google.common.collect.Lists; +import org.apache.drill.common.util.DrillVersionInfo; +import org.apache.hadoop.fs.Path; + +import java.util.List; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.SUPPORTED_VERSIONS; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetFileMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTableMetadata_v3; + +/** + * Util class that contains helper methods for converting paths in the table and directory metadata structures + */ +public class MetadataPathUtils { + + /** + * Helper method that converts a list of relative paths to absolute ones + * + * @param paths list of relative paths + * @param baseDir base parent directory + * @return list of absolute paths + */ + public static List convertToAbsolutePaths(List paths, String baseDir) { + if (!paths.isEmpty()) { + List absolutePaths = Lists.newArrayList(); + for (String relativePath : paths) { + String absolutePath = (new Path(relativePath).isAbsolute()) ? relativePath + : new Path(baseDir, relativePath).toUri().getPath(); + absolutePaths.add(absolutePath); + } + return absolutePaths; + } + return paths; + } + + /** + * Convert a list of files with relative paths to files with absolute ones + * + * @param files list of files with relative paths + * @param baseDir base parent directory + * @return list of files with absolute paths + */ + public static List convertToFilesWithAbsolutePaths( + List files, String baseDir) { + if (!files.isEmpty()) { + List filesWithAbsolutePaths = Lists.newArrayList(); + for (ParquetFileMetadata_v3 file : files) { + Path relativePath = new Path(file.getPath()); + // create a new file if old one contains a relative path, otherwise use an old file + ParquetFileMetadata_v3 fileWithAbsolutePath = (relativePath.isAbsolute()) ? file + : new ParquetFileMetadata_v3(new Path(baseDir, relativePath).toUri().getPath(), file.length, file.rowGroups); + filesWithAbsolutePaths.add(fileWithAbsolutePath); + } + return filesWithAbsolutePaths; + } + return files; + } + + /** + * Creates a new parquet table metadata from the {@code tableMetadataWithAbsolutePaths} parquet table. + * A new parquet table will contain relative paths for the files and directories. + * + * @param tableMetadataWithAbsolutePaths parquet table metadata with absolute paths for the files and directories + * @param baseDir base parent directory + * @return parquet table metadata with relative paths for the files and directories + */ + public static ParquetTableMetadata_v3 createMetadataWithRelativePaths( + ParquetTableMetadata_v3 tableMetadataWithAbsolutePaths, String baseDir) { + List directoriesWithRelativePaths = Lists.newArrayList(); + for (String directory : tableMetadataWithAbsolutePaths.getDirectories()) { + directoriesWithRelativePaths.add(relativize(baseDir, directory)) ; + } + List filesWithRelativePaths = Lists.newArrayList(); + for (ParquetFileMetadata_v3 file : tableMetadataWithAbsolutePaths.files) { + filesWithRelativePaths.add(new ParquetFileMetadata_v3( + relativize(baseDir, file.getPath()), file.length, file.rowGroups)); + } + return new ParquetTableMetadata_v3(SUPPORTED_VERSIONS.last().toString(), tableMetadataWithAbsolutePaths, + filesWithRelativePaths, directoriesWithRelativePaths, DrillVersionInfo.getVersion()); + } + + /** + * Constructs relative path from child full path and base path. Or return child path if the last one is already relative + * + * @param childPath full absolute path + * @param baseDir base path (the part of the Path, which should be cut off from child path) + * @return relative path + */ + public static String relativize(String baseDir, String childPath) { + Path fullPathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(childPath)); + Path basePathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(baseDir)); + + // Since hadoop Path hasn't relativize() we use uri.relativize() to get relative path + Path relativeFilePath = new Path(basePathWithoutSchemeAndAuthority.toUri() + .relativize(fullPathWithoutSchemeAndAuthority.toUri())); + if (relativeFilePath.isAbsolute()) { + throw new IllegalStateException(String.format("Path %s is not a subpath of %s.", + basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath())); + } + return relativeFilePath.toUri().getPath(); + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataVersion.java similarity index 99% rename from exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java rename to exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataVersion.java index 7abb9f13b6d..15b4b9da3a5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/MetadataVersion.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.drill.exec.store.parquet; +package org.apache.drill.exec.store.parquet.metadata; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -26,7 +26,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; - public class MetadataVersion implements Comparable { private static final String FORMAT = "v?((?!0)\\d+)(\\.(\\d+))?"; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java new file mode 100644 index 00000000000..92feb5f8ab3 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V1.java @@ -0,0 +1,329 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V1; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.RowGroupMetadata; + + +public class Metadata_V1 { + + @JsonTypeName(V1) + public static class ParquetTableMetadata_v1 extends ParquetTableMetadataBase { + @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion; + @JsonProperty + List files; + @JsonProperty List directories; + + public ParquetTableMetadata_v1() { + } + + public ParquetTableMetadata_v1(String metadataVersion, List files, List directories) { + this.metadataVersion = metadataVersion; + this.files = files; + this.directories = directories; + } + + @JsonIgnore + @Override public List getDirectories() { + return directories; + } + + @JsonIgnore @Override public List getFiles() { + return files; + } + + @JsonIgnore @Override public void assignFiles(List newFiles) { + this.files = (List) newFiles; + } + + @Override public boolean hasColumnMetadata() { + return false; + } + + @JsonIgnore @Override public PrimitiveType.PrimitiveTypeName getPrimitiveType(String[] columnName) { + return null; + } + + @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) { + return null; + } + + @JsonIgnore @Override + public Integer getRepetitionLevel(String[] columnName) { + return null; + } + + @JsonIgnore @Override + public Integer getDefinitionLevel(String[] columnName) { + return null; + } + + @JsonIgnore @Override + public boolean isRowGroupPrunable() { + return false; + } + + @JsonIgnore @Override public MetadataBase.ParquetTableMetadataBase clone() { + return new ParquetTableMetadata_v1(metadataVersion, files, directories); + } + + @JsonIgnore @Override + public String getDrillVersion() { + return null; + } + + @JsonIgnore @Override public String getMetadataVersion() { + return metadataVersion; + } + } + + + /** + * Struct which contains the metadata for a single parquet file + */ + public static class ParquetFileMetadata_v1 extends ParquetFileMetadata { + @JsonProperty + public String path; + @JsonProperty + public Long length; + @JsonProperty + public List rowGroups; + + public ParquetFileMetadata_v1() { + } + + public ParquetFileMetadata_v1(String path, Long length, List rowGroups) { + this.path = path; + this.length = length; + this.rowGroups = rowGroups; + } + + @Override + public String toString() { + return String.format("path: %s rowGroups: %s", path, rowGroups); + } + + @JsonIgnore @Override public String getPath() { + return path; + } + + @JsonIgnore @Override public Long getLength() { + return length; + } + + @JsonIgnore @Override public List getRowGroups() { + return rowGroups; + } + } + + + /** + * A struct that contains the metadata for a parquet row group + */ + public static class RowGroupMetadata_v1 extends RowGroupMetadata { + @JsonProperty + public Long start; + @JsonProperty + public Long length; + @JsonProperty + public Long rowCount; + @JsonProperty + public Map hostAffinity; + @JsonProperty + public List columns; + + public RowGroupMetadata_v1() { + } + + public RowGroupMetadata_v1(Long start, Long length, Long rowCount, Map hostAffinity, + List columns) { + this.start = start; + this.length = length; + this.rowCount = rowCount; + this.hostAffinity = hostAffinity; + this.columns = columns; + } + + @Override public Long getStart() { + return start; + } + + @Override public Long getLength() { + return length; + } + + @Override public Long getRowCount() { + return rowCount; + } + + @Override public Map getHostAffinity() { + return hostAffinity; + } + + @Override public List getColumns() { + return columns; + } + } + + + /** + * A struct that contains the metadata for a column in a parquet file + */ + public static class ColumnMetadata_v1 extends ColumnMetadata { + @JsonProperty + public SchemaPath name; + @JsonProperty + public PrimitiveType.PrimitiveTypeName primitiveType; + @JsonProperty + public OriginalType originalType; + @JsonProperty + public Long nulls; + + // JsonProperty for these are associated with the getters and setters + public Object max; + public Object min; + + + public ColumnMetadata_v1() { + } + + public ColumnMetadata_v1(SchemaPath name, PrimitiveType.PrimitiveTypeName primitiveType, OriginalType originalType, + Object max, Object min, Long nulls) { + this.name = name; + this.primitiveType = primitiveType; + this.originalType = originalType; + this.max = max; + this.min = min; + this.nulls = nulls; + } + + @JsonProperty(value = "min") + public Object getMin() { + if (primitiveType == PrimitiveType.PrimitiveTypeName.BINARY && min != null) { + return new String(((Binary) min).getBytes()); + } + return min; + } + + @JsonProperty(value = "max") + public Object getMax() { + if (primitiveType == PrimitiveType.PrimitiveTypeName.BINARY && max != null) { + return new String(((Binary) max).getBytes()); + } + return max; + } + + @Override public PrimitiveType.PrimitiveTypeName getPrimitiveType() { + return primitiveType; + } + + @Override public OriginalType getOriginalType() { + return originalType; + } + + /** + * setter used during deserialization of the 'min' field of the metadata cache file. + * + * @param min + */ + @JsonProperty(value = "min") + public void setMin(Object min) { + this.min = min; + } + + /** + * setter used during deserialization of the 'max' field of the metadata cache file. + * + * @param max + */ + @JsonProperty(value = "max") + public void setMax(Object max) { + this.max = max; + } + + @Override public String[] getName() { + String[] s = new String[1]; + String nameString = name.toString(); + // Strip out the surrounding backticks. + s[0]=nameString.substring(1, nameString.length()-1); + return s; + } + + @Override public Long getNulls() { + return nulls; + } + + /** + * Checks that the column chunk has a single value. + * Returns {@code true} if {@code min} and {@code max} are the same but not null + * and nulls count is 0 or equal to the rows count. + *

+ * Returns {@code true} if {@code min} and {@code max} are null and the number of null values + * in the column chunk is equal to the rows count. + *

+ * Comparison of nulls and rows count is needed for the cases: + *

    + *
  • column with primitive type has single value and null values
  • + * + *
  • column with primitive type has only null values, min/max couldn't be null, + * but column has single value
  • + *
+ * + * @param rowCount rows count in column chunk + * @return true if column has single value + */ + @Override + public boolean hasSingleValue(long rowCount) { + if (nulls != null) { + if (min != null) { + // Objects.deepEquals() is used here, since min and max may be byte arrays + return Objects.deepEquals(min, max) && (nulls == 0 || nulls == rowCount); + } else { + return nulls == rowCount && max == null; + } + } + return false; + } + + @Override public Object getMinValue() { + return min; + } + + @Override public Object getMaxValue() { + return max; + } + + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java new file mode 100644 index 00000000000..7eddc127994 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V2.java @@ -0,0 +1,418 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V2; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.RowGroupMetadata; + +public class Metadata_V2 { + + /** + * Struct which contains the metadata for an entire parquet directory structure + */ + @JsonTypeName(V2) public static class ParquetTableMetadata_v2 extends ParquetTableMetadataBase { + @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion; + /* + ColumnTypeInfo is schema information from all the files and row groups, merged into + one. To get this info, we pass the ParquetTableMetadata object all the way dow to the + RowGroup and the column type is built there as it is read from the footer. + */ + @JsonProperty public ConcurrentHashMap columnTypeInfo; + @JsonProperty + List files; + @JsonProperty List directories; + @JsonProperty String drillVersion; + + public ParquetTableMetadata_v2() { + } + + public ParquetTableMetadata_v2(String metadataVersion, String drillVersion) { + this.metadataVersion = metadataVersion; + this.drillVersion = drillVersion; + } + + public ParquetTableMetadata_v2(String metadataVersion, ParquetTableMetadataBase parquetTable, + List files, List directories, String drillVersion) { + this.metadataVersion = metadataVersion; + this.files = files; + this.directories = directories; + this.columnTypeInfo = ((ParquetTableMetadata_v2) parquetTable).columnTypeInfo; + this.drillVersion = drillVersion; + } + + public ParquetTableMetadata_v2(String metadataVersion, List files, List directories, + ConcurrentHashMap columnTypeInfo, String drillVersion) { + this.metadataVersion = metadataVersion; + this.files = files; + this.directories = directories; + this.columnTypeInfo = columnTypeInfo; + this.drillVersion = drillVersion; + } + + public ColumnTypeMetadata_v2 getColumnTypeInfo(String[] name) { + return columnTypeInfo.get(new ColumnTypeMetadata_v2.Key(name)); + } + + @JsonIgnore + @Override public List getDirectories() { + return directories; + } + + @JsonIgnore @Override public List getFiles() { + return files; + } + + @JsonIgnore @Override public void assignFiles(List newFiles) { + this.files = (List) newFiles; + } + + @Override public boolean hasColumnMetadata() { + return true; + } + + @JsonIgnore @Override public PrimitiveType.PrimitiveTypeName getPrimitiveType(String[] columnName) { + return getColumnTypeInfo(columnName).primitiveType; + } + + @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) { + return getColumnTypeInfo(columnName).originalType; + } + + @JsonIgnore @Override + public Integer getRepetitionLevel(String[] columnName) { + return null; + } + + @JsonIgnore @Override + public Integer getDefinitionLevel(String[] columnName) { + return null; + } + + @JsonIgnore @Override + public boolean isRowGroupPrunable() { + return false; + } + + @JsonIgnore @Override public ParquetTableMetadataBase clone() { + return new ParquetTableMetadata_v2(metadataVersion, files, directories, columnTypeInfo, drillVersion); + } + + @JsonIgnore @Override + public String getDrillVersion() { + return drillVersion; + } + + @JsonIgnore @Override public String getMetadataVersion() { + return metadataVersion; + } + + } + + + /** + * Struct which contains the metadata for a single parquet file + */ + public static class ParquetFileMetadata_v2 extends ParquetFileMetadata { + @JsonProperty public String path; + @JsonProperty public Long length; + @JsonProperty public List rowGroups; + + public ParquetFileMetadata_v2() { + } + + public ParquetFileMetadata_v2(String path, Long length, List rowGroups) { + this.path = path; + this.length = length; + this.rowGroups = rowGroups; + } + + @Override public String toString() { + return String.format("path: %s rowGroups: %s", path, rowGroups); + } + + @JsonIgnore @Override public String getPath() { + return path; + } + + @JsonIgnore @Override public Long getLength() { + return length; + } + + @JsonIgnore @Override public List getRowGroups() { + return rowGroups; + } + } + + + /** + * A struct that contains the metadata for a parquet row group + */ + public static class RowGroupMetadata_v2 extends RowGroupMetadata { + @JsonProperty public Long start; + @JsonProperty public Long length; + @JsonProperty public Long rowCount; + @JsonProperty public Map hostAffinity; + @JsonProperty public List columns; + + public RowGroupMetadata_v2() { + } + + public RowGroupMetadata_v2(Long start, Long length, Long rowCount, Map hostAffinity, + List columns) { + this.start = start; + this.length = length; + this.rowCount = rowCount; + this.hostAffinity = hostAffinity; + this.columns = columns; + } + + @Override public Long getStart() { + return start; + } + + @Override public Long getLength() { + return length; + } + + @Override public Long getRowCount() { + return rowCount; + } + + @Override public Map getHostAffinity() { + return hostAffinity; + } + + @Override public List getColumns() { + return columns; + } + } + + + public static class ColumnTypeMetadata_v2 { + @JsonProperty public String[] name; + @JsonProperty public PrimitiveType.PrimitiveTypeName primitiveType; + @JsonProperty public OriginalType originalType; + + // Key to find by name only + @JsonIgnore private Key key; + + public ColumnTypeMetadata_v2() { + } + + public ColumnTypeMetadata_v2(String[] name, PrimitiveType.PrimitiveTypeName primitiveType, OriginalType originalType) { + this.name = name; + this.primitiveType = primitiveType; + this.originalType = originalType; + this.key = new Key(name); + } + + @JsonIgnore private Key key() { + return this.key; + } + + public static class Key { + private String[] name; + private int hashCode = 0; + + public Key(String[] name) { + this.name = name; + } + + @Override public int hashCode() { + if (hashCode == 0) { + hashCode = Arrays.hashCode(name); + } + return hashCode; + } + + @Override public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final Key other = (Key) obj; + return Arrays.equals(this.name, other.name); + } + + @Override public String toString() { + String s = null; + for (String namePart : name) { + if (s != null) { + s += "."; + s += namePart; + } else { + s = namePart; + } + } + return s; + } + + public static class DeSerializer extends KeyDeserializer { + + public DeSerializer() { + } + + @Override + public Object deserializeKey(String key, com.fasterxml.jackson.databind.DeserializationContext ctxt) + throws IOException, com.fasterxml.jackson.core.JsonProcessingException { + return new Key(key.split("\\.")); + } + } + } + } + + + /** + * A struct that contains the metadata for a column in a parquet file + */ + public static class ColumnMetadata_v2 extends ColumnMetadata { + // Use a string array for name instead of Schema Path to make serialization easier + @JsonProperty public String[] name; + @JsonProperty public Long nulls; + + public Object mxValue; + + @JsonIgnore private PrimitiveType.PrimitiveTypeName primitiveType; + + public ColumnMetadata_v2() { + } + + public ColumnMetadata_v2(String[] name, PrimitiveType.PrimitiveTypeName primitiveType, Object mxValue, Long nulls) { + this.name = name; + this.mxValue = mxValue; + this.nulls = nulls; + this.primitiveType = primitiveType; + } + + @JsonProperty(value = "mxValue") public void setMax(Object mxValue) { + this.mxValue = mxValue; + } + + @Override public String[] getName() { + return name; + } + + @Override public Long getNulls() { + return nulls; + } + + /** + * Checks that the column chunk has a single value. + * Returns {@code true} if {@code mxValue} is not null + * and nulls count is 0 or if nulls count is equal to the rows count. + *

+ * Comparison of nulls and rows count is needed for the cases: + *

    + *
  • column with primitive type has single value and null values
  • + * + *
  • column with binary type has only null values, so column has single value
  • + *
+ * + * @param rowCount rows count in column chunk + * @return true if column has single value + */ + @Override + public boolean hasSingleValue(long rowCount) { + return (mxValue != null && nulls == 0) || nulls == rowCount; + } + + @Override public Object getMinValue() { + return mxValue; + } + + @Override public Object getMaxValue() { + return mxValue; + } + + @Override + public void setMin(Object newMin) { + // noop - min value not stored in this version of the metadata + } + + @Override public PrimitiveType.PrimitiveTypeName getPrimitiveType() { + return primitiveType; + } + + @Override public OriginalType getOriginalType() { + return null; + } + + public static class DeSerializer extends JsonDeserializer { + @Override public ColumnMetadata_v2 deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException, JsonProcessingException { + return null; + } + } + + + // We use a custom serializer and write only non null values. + public static class Serializer extends JsonSerializer { + @Override + public void serialize(ColumnMetadata_v2 value, JsonGenerator jgen, SerializerProvider provider) + throws IOException, JsonProcessingException { + jgen.writeStartObject(); + jgen.writeArrayFieldStart("name"); + for (String n : value.name) { + jgen.writeString(n); + } + jgen.writeEndArray(); + if (value.mxValue != null) { + Object val; + if (value.primitiveType == PrimitiveType.PrimitiveTypeName.BINARY && value.mxValue != null) { + val = new String(((Binary) value.mxValue).getBytes()); + } else { + val = value.mxValue; + } + jgen.writeObjectField("mxValue", val); + } + if (value.nulls != null) { + jgen.writeObjectField("nulls", value.nulls); + } + jgen.writeEndObject(); + } + } + + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java new file mode 100644 index 00000000000..f378a703c44 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata_V3.java @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.drill.common.expression.SchemaPath; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.drill.exec.store.parquet.metadata.MetadataVersion.Constants.V3_3; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetFileMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.RowGroupMetadata; + +public class Metadata_V3 { + + @JsonTypeName(V3_3) + public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase { + @JsonProperty(value = "metadata_version", access = JsonProperty.Access.WRITE_ONLY) private String metadataVersion; + /* + ColumnTypeInfo is schema information from all the files and row groups, merged into + one. To get this info, we pass the ParquetTableMetadata object all the way dow to the + RowGroup and the column type is built there as it is read from the footer. + */ + @JsonProperty public ConcurrentHashMap columnTypeInfo; + @JsonProperty + List files; + @JsonProperty List directories; + @JsonProperty String drillVersion; + + /** + * Default constructor needed for deserialization from Parquet Metadata Cache Files + */ + public ParquetTableMetadata_v3() { + } + + /** + * Used for creating the Parquet Metadata cache file + * @param metadataVersion metadata version + * @param drillVersion apache drill version + */ + public ParquetTableMetadata_v3(String metadataVersion, String drillVersion) { + this.metadataVersion = metadataVersion; + this.drillVersion = drillVersion; + } + + public ParquetTableMetadata_v3(String metadataVersion, ParquetTableMetadataBase parquetTable, + List files, List directories, String drillVersion) { + this.metadataVersion = metadataVersion; + this.files = files; + this.directories = directories; + this.columnTypeInfo = ((ParquetTableMetadata_v3) parquetTable).columnTypeInfo; + this.drillVersion = drillVersion; + } + + public ParquetTableMetadata_v3(String metadataVersion, List files, List directories, + ConcurrentHashMap columnTypeInfo, + String drillVersion) { + this.metadataVersion = metadataVersion; + this.files = files; + this.directories = directories; + this.columnTypeInfo = columnTypeInfo; + this.drillVersion = drillVersion; + } + + public ColumnTypeMetadata_v3 getColumnTypeInfo(String[] name) { + return columnTypeInfo.get(new ColumnTypeMetadata_v3.Key(name)); + } + + @JsonIgnore + @Override public List getDirectories() { + return directories; + } + + @JsonIgnore @Override public String getMetadataVersion() { + return metadataVersion; + } + + /** + * If directories list and file metadata list contain relative paths, update it to absolute ones + * @param baseDir base parent directory + */ + @JsonIgnore public void updateRelativePaths(String baseDir) { + // update directories paths to absolute ones + this.directories = MetadataPathUtils.convertToAbsolutePaths(directories, baseDir); + + // update files paths to absolute ones + this.files = MetadataPathUtils.convertToFilesWithAbsolutePaths(files, baseDir); + } + + @JsonIgnore @Override public List getFiles() { + return files; + } + + @JsonIgnore @Override public void assignFiles(List newFiles) { + this.files = (List) newFiles; + } + + @Override public boolean hasColumnMetadata() { + return true; + } + + @JsonIgnore @Override public PrimitiveType.PrimitiveTypeName getPrimitiveType(String[] columnName) { + return getColumnTypeInfo(columnName).primitiveType; + } + + @JsonIgnore @Override public OriginalType getOriginalType(String[] columnName) { + return getColumnTypeInfo(columnName).originalType; + } + + @JsonIgnore @Override + public Integer getRepetitionLevel(String[] columnName) { + return getColumnTypeInfo(columnName).repetitionLevel; + } + + @JsonIgnore @Override + public Integer getDefinitionLevel(String[] columnName) { + return getColumnTypeInfo(columnName).definitionLevel; + } + + @JsonIgnore @Override + public boolean isRowGroupPrunable() { + return true; + } + + @JsonIgnore @Override public ParquetTableMetadataBase clone() { + return new ParquetTableMetadata_v3(metadataVersion, files, directories, columnTypeInfo, drillVersion); + } + + @JsonIgnore @Override + public String getDrillVersion() { + return drillVersion; + } + + } + + + /** + * Struct which contains the metadata for a single parquet file + */ + public static class ParquetFileMetadata_v3 extends ParquetFileMetadata { + @JsonProperty public String path; + @JsonProperty public Long length; + @JsonProperty public List rowGroups; + + public ParquetFileMetadata_v3() { + } + + public ParquetFileMetadata_v3(String path, Long length, List rowGroups) { + this.path = path; + this.length = length; + this.rowGroups = rowGroups; + } + + @Override public String toString() { + return String.format("path: %s rowGroups: %s", path, rowGroups); + } + + @JsonIgnore @Override public String getPath() { + return path; + } + + @JsonIgnore @Override public Long getLength() { + return length; + } + + @JsonIgnore @Override public List getRowGroups() { + return rowGroups; + } + } + + + /** + * A struct that contains the metadata for a parquet row group + */ + public static class RowGroupMetadata_v3 extends RowGroupMetadata { + @JsonProperty public Long start; + @JsonProperty public Long length; + @JsonProperty public Long rowCount; + @JsonProperty public Map hostAffinity; + @JsonProperty public List columns; + + public RowGroupMetadata_v3() { + } + + public RowGroupMetadata_v3(Long start, Long length, Long rowCount, Map hostAffinity, + List columns) { + this.start = start; + this.length = length; + this.rowCount = rowCount; + this.hostAffinity = hostAffinity; + this.columns = columns; + } + + @Override public Long getStart() { + return start; + } + + @Override public Long getLength() { + return length; + } + + @Override public Long getRowCount() { + return rowCount; + } + + @Override public Map getHostAffinity() { + return hostAffinity; + } + + @Override public List getColumns() { + return columns; + } + } + + + public static class ColumnTypeMetadata_v3 { + @JsonProperty public String[] name; + @JsonProperty public PrimitiveType.PrimitiveTypeName primitiveType; + @JsonProperty public OriginalType originalType; + @JsonProperty public int precision; + @JsonProperty public int scale; + @JsonProperty public int repetitionLevel; + @JsonProperty public int definitionLevel; + + // Key to find by name only + @JsonIgnore private Key key; + + public ColumnTypeMetadata_v3() { + } + + public ColumnTypeMetadata_v3(String[] name, PrimitiveType.PrimitiveTypeName primitiveType, OriginalType originalType, int precision, int scale, int repetitionLevel, int definitionLevel) { + this.name = name; + this.primitiveType = primitiveType; + this.originalType = originalType; + this.precision = precision; + this.scale = scale; + this.repetitionLevel = repetitionLevel; + this.definitionLevel = definitionLevel; + this.key = new Key(name); + } + + @JsonIgnore private Key key() { + return this.key; + } + + public static class Key { + private SchemaPath name; + private int hashCode = 0; + + public Key(String[] name) { + this.name = SchemaPath.getCompoundPath(name); + } + + public Key(SchemaPath name) { + this.name = new SchemaPath(name); + } + + @Override public int hashCode() { + if (hashCode == 0) { + hashCode = name.hashCode(); + } + return hashCode; + } + + @Override public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final Key other = (Key) obj; + return this.name.equals(other.name); + } + + @Override public String toString() { + return name.toString(); + } + + public static class DeSerializer extends KeyDeserializer { + + public DeSerializer() { + } + + @Override + public Object deserializeKey(String key, com.fasterxml.jackson.databind.DeserializationContext ctxt) + throws IOException, com.fasterxml.jackson.core.JsonProcessingException { + // key string should contain '`' char if the field was serialized as SchemaPath object + if (key.contains("`")) { + return new Key(SchemaPath.parseFromString(key)); + } + return new Key(key.split("\\.")); + } + } + } + } + + + /** + * A struct that contains the metadata for a column in a parquet file + */ + public static class ColumnMetadata_v3 extends ColumnMetadata { + // Use a string array for name instead of Schema Path to make serialization easier + @JsonProperty public String[] name; + @JsonProperty public Long nulls; + + public Object minValue; + public Object maxValue; + + @JsonIgnore private PrimitiveType.PrimitiveTypeName primitiveType; + + public ColumnMetadata_v3() { + } + + public ColumnMetadata_v3(String[] name, PrimitiveType.PrimitiveTypeName primitiveType, Object minValue, Object maxValue, Long nulls) { + this.name = name; + this.minValue = minValue; + this.maxValue = maxValue; + this.nulls = nulls; + this.primitiveType = primitiveType; + } + + @JsonProperty(value = "minValue") public void setMin(Object minValue) { + this.minValue = minValue; + } + + @JsonProperty(value = "maxValue") public void setMax(Object maxValue) { + this.maxValue = maxValue; + } + + @Override public String[] getName() { + return name; + } + + @Override public Long getNulls() { + return nulls; + } + + /** + * Checks that the column chunk has a single value. + * Returns {@code true} if {@code minValue} and {@code maxValue} are the same but not null + * and nulls count is 0 or equal to the rows count. + *

+ * Returns {@code true} if {@code minValue} and {@code maxValue} are null and the number of null values + * in the column chunk is equal to the rows count. + *

+ * Comparison of nulls and rows count is needed for the cases: + *

    + *
  • column with primitive type has single value and null values
  • + * + *
  • column with primitive type has only null values, min/max couldn't be null, + * but column has single value
  • + *
+ * + * @param rowCount rows count in column chunk + * @return true if column has single value + */ + @Override + public boolean hasSingleValue(long rowCount) { + if (nulls != null) { + if (minValue != null) { + // Objects.deepEquals() is used here, since min and max may be byte arrays + return Objects.deepEquals(minValue, maxValue) && (nulls == 0 || nulls == rowCount); + } else { + return nulls == rowCount && maxValue == null; + } + } + return false; + } + + @Override public Object getMinValue() { + return minValue; + } + + @Override public Object getMaxValue() { + return maxValue; + } + + @Override public PrimitiveType.PrimitiveTypeName getPrimitiveType() { + return null; + } + + @Override public OriginalType getOriginalType() { + return null; + } + + public static class DeSerializer extends JsonDeserializer { + @Override public ColumnMetadata_v3 deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException, JsonProcessingException { + return null; + } + } + + + // We use a custom serializer and write only non null values. + public static class Serializer extends JsonSerializer { + @Override + public void serialize(ColumnMetadata_v3 value, JsonGenerator jgen, SerializerProvider provider) + throws IOException, JsonProcessingException { + jgen.writeStartObject(); + jgen.writeArrayFieldStart("name"); + for (String n : value.name) { + jgen.writeString(n); + } + jgen.writeEndArray(); + if (value.minValue != null) { + Object val; + if (value.primitiveType == PrimitiveType.PrimitiveTypeName.BINARY + || value.primitiveType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { + val = ((Binary) value.minValue).getBytes(); + } else { + val = value.minValue; + } + jgen.writeObjectField("minValue", val); + } + if (value.maxValue != null) { + Object val; + if (value.primitiveType == PrimitiveType.PrimitiveTypeName.BINARY + || value.primitiveType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { + val = ((Binary) value.maxValue).getBytes(); + } else { + val = value.maxValue; + } + jgen.writeObjectField("maxValue", val); + } + + if (value.nulls != null) { + jgen.writeObjectField("nulls", value.nulls); + } + jgen.writeEndObject(); + } + } + + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java new file mode 100644 index 00000000000..186f53415f3 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/ParquetTableMetadataDirs.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.parquet.metadata; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public 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; + } + + /** If directories list contains relative paths, update it to absolute ones + * @param baseDir base parent directory + */ + @JsonIgnore public void updateRelativePaths(String baseDir) { + this.directories = MetadataPathUtils.convertToAbsolutePaths(directories, baseDir); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java index a8c12187e7f..933d8ee0eb8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/stat/ParquetMetaStatCollector.java @@ -21,8 +21,7 @@ import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos; import org.apache.drill.common.types.Types; -import org.apache.drill.exec.store.parquet.Metadata; -import org.apache.drill.exec.store.parquet.ParquetGroupScan; +import org.apache.drill.exec.store.parquet.ParquetReaderUtility; import org.apache.parquet.column.statistics.BinaryStatistics; import org.apache.parquet.column.statistics.BooleanStatistics; import org.apache.parquet.column.statistics.DoubleStatistics; @@ -40,15 +39,20 @@ import java.util.Set; import java.util.concurrent.TimeUnit; -public class ParquetMetaStatCollector implements ColumnStatCollector{ +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ColumnTypeMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.Metadata_V3.ParquetTableMetadata_v3; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ColumnMetadata; +import static org.apache.drill.exec.store.parquet.metadata.MetadataBase.ParquetTableMetadataBase; + +public class ParquetMetaStatCollector implements ColumnStatCollector { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetMetaStatCollector.class); - private final Metadata.ParquetTableMetadataBase parquetTableMetadata; - private final List columnMetadataList; + private final ParquetTableMetadataBase parquetTableMetadata; + private final List columnMetadataList; private final Map implicitColValues; - public ParquetMetaStatCollector(Metadata.ParquetTableMetadataBase parquetTableMetadata, - List columnMetadataList, Map implicitColValues) { + public ParquetMetaStatCollector(ParquetTableMetadataBase parquetTableMetadata, + List columnMetadataList, Map implicitColValues) { this.parquetTableMetadata = parquetTableMetadata; this.columnMetadataList = columnMetadataList; @@ -69,15 +73,15 @@ public ParquetMetaStatCollector(Metadata.ParquetTableMetadataBase parquetTableMe @Override public Map collectColStat(Set fields) { - Stopwatch timer = Stopwatch.createStarted(); + Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null; // map from column to ColumnMetadata - final Map columnMetadataMap = new HashMap<>(); + final Map columnMetadataMap = new HashMap<>(); // map from column name to column statistics. final Map statMap = new HashMap<>(); - for (final Metadata.ColumnMetadata columnMetadata : columnMetadataList) { + for (final ColumnMetadata columnMetadata : columnMetadataList) { SchemaPath schemaPath = SchemaPath.getCompoundPath(columnMetadata.getName()); columnMetadataMap.put(schemaPath, columnMetadata); } @@ -86,7 +90,7 @@ public Map collectColStat(Set fields) final PrimitiveType.PrimitiveTypeName primitiveType; final OriginalType originalType; - final Metadata.ColumnMetadata columnMetadata = columnMetadataMap.get(field.getUnIndexed()); + final ColumnMetadata columnMetadata = columnMetadataMap.get(field.getUnIndexed()); if (columnMetadata != null) { final Object min = columnMetadata.getMinValue(); @@ -98,8 +102,8 @@ public Map collectColStat(Set fields) int precision = 0; int scale = 0; // ColumnTypeMetadata_v3 stores information about scale and precision - if (parquetTableMetadata instanceof Metadata.ParquetTableMetadata_v3) { - Metadata.ColumnTypeMetadata_v3 columnTypeInfo = ((Metadata.ParquetTableMetadata_v3) parquetTableMetadata) + if (parquetTableMetadata instanceof ParquetTableMetadata_v3) { + ColumnTypeMetadata_v3 columnTypeInfo = ((ParquetTableMetadata_v3) parquetTableMetadata) .getColumnTypeInfo(columnMetadata.getName()); scale = columnTypeInfo.scale; precision = columnTypeInfo.precision; @@ -119,8 +123,9 @@ public Map collectColStat(Set fields) } } - if (logger.isDebugEnabled()) { + if (timer != null) { logger.debug("Took {} ms to column statistics for row group", timer.elapsed(TimeUnit.MILLISECONDS)); + timer.stop(); } return statMap; @@ -145,7 +150,7 @@ private ColumnStatistics getStat(Object min, Object max, Long numNull, Statistics stat = Statistics.getStatsBasedOnType(primitiveType); Statistics convertedStat = stat; - TypeProtos.MajorType type = ParquetGroupScan.getType(primitiveType, originalType, scale, precision); + TypeProtos.MajorType type = ParquetReaderUtility.getType(primitiveType, originalType, scale, precision); if (numNull != null) { stat.setNumNulls(numNull); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java index 95820f24d5b..aa9ead3c853 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java @@ -25,7 +25,7 @@ import org.apache.drill.categories.UnlikelyTest; import org.apache.drill.common.exceptions.UserRemoteException; import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.store.parquet.Metadata; +import org.apache.drill.exec.store.parquet.metadata.Metadata; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.BeforeClass; diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java index 7c781ac7ec0..20438cdedde 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java @@ -18,21 +18,45 @@ package org.apache.drill.exec.store; import org.apache.drill.PlanTestBase; +import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.store.avro.AvroTestUtil; import org.junit.Test; import java.nio.file.Paths; +import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_TMP_SCHEMA; + public class FormatPluginSerDeTest extends PlanTestBase { @Test public void testParquet() throws Exception { - test("alter session set `planner.slice_target` = 1"); - testPhysicalPlanSubmission( - String.format("select * from table(cp.`%s`(type=>'parquet'))", "parquet/alltypes_required.parquet"), - String.format("select * from table(cp.`%s`(type=>'parquet', autoCorrectCorruptDates=>false))", "parquet/alltypes_required.parquet"), - String.format("select * from table(cp.`%s`(type=>'parquet', autoCorrectCorruptDates=>true))", "parquet/alltypes_required.parquet") - ); + try { + setSessionOption(ExecConstants.SLICE_TARGET, 1); + testPhysicalPlanSubmission( + String.format("select * from table(cp.`%s`(type=>'parquet'))", "parquet/alltypes_required.parquet"), + String.format("select * from table(cp.`%s`(type=>'parquet', autoCorrectCorruptDates=>false))", "parquet/alltypes_required.parquet"), + String.format("select * from table(cp.`%s`(type=>'parquet', autoCorrectCorruptDates=>true))", "parquet/alltypes_required.parquet")); + } finally { + resetSessionOption(ExecConstants.SLICE_TARGET); + } + } + + @Test + public void testParquetWithMetadata() throws Exception { + String tableName = "alltypes_required_with_metadata"; + test("use %s", DFS_TMP_SCHEMA); + try { + test("create table %s as select * from cp.`parquet/alltypes_required.parquet`", tableName); + test("refresh table metadata %s", tableName); + setSessionOption(ExecConstants.SLICE_TARGET, 1); + testPhysicalPlanSubmission( + String.format("select * from table(`%s`(type=>'parquet'))", tableName), + String.format("select * from table(`%s`(type=>'parquet', autoCorrectCorruptDates=>false))", tableName), + String.format("select * from table(`%s`(type=>'parquet', autoCorrectCorruptDates=>true))", tableName)); + } finally { + resetSessionOption(ExecConstants.SLICE_TARGET); + test("drop table if exists %s", tableName); + } } @Test 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 7bccb332302..50e679ad64f 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 @@ -24,6 +24,8 @@ import org.apache.drill.categories.UnlikelyTest; import org.apache.commons.io.FileUtils; import org.apache.drill.exec.record.BatchSchema; +import org.apache.drill.exec.store.parquet.metadata.Metadata; +import org.apache.drill.exec.store.parquet.metadata.MetadataVersion; import org.apache.drill.test.rowSet.schema.SchemaBuilder; import org.apache.drill.exec.planner.physical.PlannerSettings; import org.junit.Assert; diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java index 4640728c89b..8de5afc6fe0 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java @@ -18,6 +18,7 @@ package org.apache.drill.exec.store.parquet; import org.apache.drill.common.exceptions.DrillRuntimeException; +import org.apache.drill.exec.store.parquet.metadata.MetadataVersion; import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java index bd528aad143..72b5eab9965 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java @@ -537,6 +537,10 @@ public enum CoreOperatorType * UNNEST = 42; */ UNNEST(42, 42), + /** + * HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN = 43; + */ + HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN(43, 43), ; /** @@ -711,6 +715,10 @@ public enum CoreOperatorType * UNNEST = 42; */ public static final int UNNEST_VALUE = 42; + /** + * HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN = 43; + */ + public static final int HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN_VALUE = 43; public final int getNumber() { return value; } @@ -760,6 +768,7 @@ public static CoreOperatorType valueOf(int value) { case 40: return FLATTEN; case 41: return LATERAL_JOIN; case 42: return UNNEST; + case 43: return HIVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN; default: return null; } } @@ -24140,7 +24149,7 @@ public Builder clearStatus() { "agmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALL" + "OCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\t" + "CANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_" + - "REQUESTED\020\006*\302\006\n\020CoreOperatorType\022\021\n\rSING" + + "REQUESTED\020\006*\360\006\n\020CoreOperatorType\022\021\n\rSING" + "LE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FIL" + "TER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004" + "\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDE" + @@ -24161,11 +24170,12 @@ public Builder clearStatus() { "P_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_" + "SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_S" + "CAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006" + - "UNNEST\020**g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000" + - "\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020" + - "\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org" + - ".apache.drill.exec.protoB\rUserBitSharedH" + - "\001" + "UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_PARQUET_RO" + + "W_GROUP_SCAN\020+*g\n\nSaslStatus\022\020\n\014SASL_UNK" + + "NOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRE" + + "SS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B" + + ".\n\033org.apache.drill.exec.protoB\rUserBitS" + + "haredH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java index 17180a053bb..fdf29b42c95 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java @@ -64,7 +64,8 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite