From c2ae7c5c0ed02294a2bf0bd61769359fa0569580 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 11 Sep 2018 15:59:20 -0700 Subject: [PATCH 1/2] Upgrade Druid to 0.13.0 --- RELEASE_NOTES.txt | 4 +- druid-handler/pom.xml | 23 +- .../hadoop/hive/druid/DruidStorageHandler.java | 92 ++++--- .../hadoop/hive/druid/DruidStorageHandlerInfo.java | 9 +- .../hive/druid/DruidStorageHandlerUtils.java | 113 ++++---- .../hadoop/hive/druid/io/DruidOutputFormat.java | 30 +- .../hive/druid/io/DruidQueryBasedInputFormat.java | 18 +- .../hadoop/hive/druid/io/DruidRecordWriter.java | 34 +-- .../hive/druid/json/KafkaSupervisorIOConfig.java | 2 +- .../hive/druid/json/KafkaSupervisorReport.java | 231 ---------------- .../druid/json/KafkaSupervisorReportPayload.java | 178 ++++++++++++ .../hive/druid/json/KafkaSupervisorSpec.java | 93 +++++-- .../druid/json/KafkaSupervisorTuningConfig.java | 157 +++++------ .../hadoop/hive/druid/json/KafkaTuningConfig.java | 301 ++++++++++++++------- .../hive/druid/security/KerberosHttpClient.java | 8 +- .../hive/druid/security/ResponseCookieHandler.java | 4 +- .../RetryIfUnauthorizedResponseHandler.java | 4 +- .../druid/serde/DruidGroupByQueryRecordReader.java | 8 +- .../hive/druid/serde/DruidQueryRecordReader.java | 18 +- .../druid/serde/DruidScanQueryRecordReader.java | 4 +- .../druid/serde/DruidSelectQueryRecordReader.java | 8 +- .../apache/hadoop/hive/druid/serde/DruidSerDe.java | 10 +- .../serde/DruidTimeseriesQueryRecordReader.java | 6 +- .../druid/serde/DruidTopNQueryRecordReader.java | 8 +- .../hive/druid/DerbyConnectorTestUtility.java | 8 +- .../apache/hadoop/hive/druid/QTestDruidSerDe.java | 4 +- .../apache/hadoop/hive/druid/QTestDruidSerDe2.java | 4 +- .../hadoop/hive/druid/TestDruidStorageHandler.java | 22 +- .../druid/TestHiveDruidQueryBasedInputFormat.java | 2 +- .../hadoop/hive/druid/serde/TestDruidSerDe.java | 20 +- .../hadoop/hive/ql/io/TestDruidRecordWriter.java | 60 ++-- itests/qtest-druid/pom.xml | 12 +- .../org/apache/hive/druid/ForkingDruidNode.java | 4 +- pom.xml | 4 +- .../druid/druidmini_expressions.q.out | 12 +- .../clientpositive/druid/druidmini_mv.q.out | 30 +- .../clientpositive/druid/druidmini_test1.q.out | 12 +- .../clientpositive/druid/druidmini_test_ts.q.out | 12 +- 38 files changed, 823 insertions(+), 746 deletions(-) delete mode 100644 druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java create mode 100644 druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReportPayload.java diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt index bde815ee8ccdda181ce72efe9303794687fa00d6..c3a33bac369c0db885163f79d7f669a574209691 100644 --- a/RELEASE_NOTES.txt +++ b/RELEASE_NOTES.txt @@ -73,7 +73,7 @@ Release Notes - Hive - Version 3.1.0 * [HIVE-19118] - Vectorization: Turning on vectorization in escape_crlf produces wrong results * [HIVE-19120] - catalog not properly set for some tables in SQL upgrade scripts * [HIVE-19131] - DecimalColumnStatsMergerTest comparison review - * [HIVE-19155] - Day time saving cause Druid inserts to fail with org.apache.hive.druid.io.druid.java.util.common.UOE: Cannot add overlapping segments + * [HIVE-19155] - Day time saving cause Druid inserts to fail with org.apache.hive.druid.org.apache.druid.java.util.common.UOE: Cannot add overlapping segments * [HIVE-19157] - Assert that Insert into Druid Table fails if the publishing of metadata by HS2 fails * [HIVE-19167] - Map data type doesn't keep the order of the key/values pairs as read (Part 2, The Sequel or SQL) * [HIVE-19168] - Ranger changes for llap commands @@ -1633,7 +1633,7 @@ Release Notes - Hive - Version 3.0.0 * [HIVE-19131] - DecimalColumnStatsMergerTest comparison review * [HIVE-19137] - orcfiledump doesn't print hive.acid.version value * [HIVE-19151] - Update expected result for some TestNegativeCliDriver tests - * [HIVE-19155] - Day time saving cause Druid inserts to fail with org.apache.hive.druid.io.druid.java.util.common.UOE: Cannot add overlapping segments + * [HIVE-19155] - Day time saving cause Druid inserts to fail with org.apache.hive.druid.org.apache.druid.java.util.common.UOE: Cannot add overlapping segments * [HIVE-19157] - Assert that Insert into Druid Table fails if the publishing of metadata by HS2 fails * [HIVE-19167] - Map data type doesn't keep the order of the key/values pairs as read (Part 2, The Sequel or SQL) * [HIVE-19168] - Ranger changes for llap commands diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml index 5b8a601131a80ffb3ffc818549badc85cba058d3..c00884ea1ce8f81131c12e51098daad2fa7615d8 100644 --- a/druid-handler/pom.xml +++ b/druid-handler/pom.xml @@ -91,7 +91,7 @@ ${joda.version} - io.druid + org.apache.druid druid-server ${druid.version} @@ -108,14 +108,14 @@ jackson-databind - io.druid + org.apache.druid druid-aws-common true - io.druid + org.apache.druid druid-processing ${druid.version} @@ -128,7 +128,7 @@ true - io.druid.extensions + org.apache.druid.extensions druid-hdfs-storage ${druid.version} @@ -155,7 +155,7 @@ - io.druid.extensions + org.apache.druid.extensions mysql-metadata-storage ${druid.version} @@ -166,7 +166,7 @@ - io.druid.extensions + org.apache.druid.extensions postgresql-metadata-storage ${druid.version} @@ -240,7 +240,7 @@ test - io.druid + org.apache.druid druid-indexing-hadoop ${druid.version} test @@ -291,8 +291,8 @@ false - io.druid - org.apache.hive.druid.io.druid + org.apache.druid + org.apache.hive.druid.org.apache.druid io.netty @@ -321,8 +321,9 @@ - io.druid:* - io.druid.extensions:* + org.apache.druid:* + org.apache.druid.extensions:* + com.metamx:* io.netty:* org.apache.calcite:* com.fasterxml.jackson.core:* diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java index cc38904b3931658efe1bb6abc4ad369a66e75d9e..d39d33fef5bd5462c8a2e6059f5a60c9310e44af 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hive.druid; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -27,39 +28,41 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; -import io.druid.data.input.impl.DimensionSchema; -import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.JSONParseSpec; -import io.druid.data.input.impl.StringInputRowParser; -import io.druid.data.input.impl.TimestampSpec; -import io.druid.java.util.common.Pair; -import io.druid.java.util.common.RetryUtils; -import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.HttpClientConfig; -import io.druid.java.util.http.client.HttpClientInit; -import io.druid.java.util.http.client.Request; -import io.druid.java.util.http.client.response.FullResponseHandler; -import io.druid.java.util.http.client.response.FullResponseHolder; -import io.druid.metadata.MetadataStorageConnectorConfig; -import io.druid.metadata.MetadataStorageTablesConfig; -import io.druid.metadata.SQLMetadataConnector; -import io.druid.metadata.storage.derby.DerbyConnector; -import io.druid.metadata.storage.derby.DerbyMetadataStorage; -import io.druid.metadata.storage.mysql.MySQLConnector; -import io.druid.metadata.storage.mysql.MySQLConnectorConfig; -import io.druid.metadata.storage.postgresql.PostgreSQLConnector; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.IndexSpec; -import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.granularity.GranularitySpec; -import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.SegmentLoadingException; -import io.druid.storage.hdfs.HdfsDataSegmentPusher; -import io.druid.storage.hdfs.HdfsDataSegmentPusherConfig; -import io.druid.timeline.DataSegment; import org.apache.commons.lang3.StringUtils; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.HttpClientConfig; +import org.apache.druid.java.util.http.client.HttpClientInit; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.FullResponseHandler; +import org.apache.druid.java.util.http.client.response.FullResponseHolder; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.storage.derby.DerbyConnector; +import org.apache.druid.metadata.storage.derby.DerbyMetadataStorage; +import org.apache.druid.metadata.storage.mysql.MySQLConnector; +import org.apache.druid.metadata.storage.mysql.MySQLConnectorConfig; +import org.apache.druid.metadata.storage.postgresql.PostgreSQLConnector; +import org.apache.druid.metadata.storage.postgresql.PostgreSQLConnectorConfig; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.storage.hdfs.HdfsDataSegmentPusher; +import org.apache.druid.storage.hdfs.HdfsDataSegmentPusherConfig; +import org.apache.druid.timeline.DataSegment; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -69,7 +72,7 @@ import org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat; import org.apache.hadoop.hive.druid.io.DruidRecordWriter; import org.apache.hadoop.hive.druid.json.KafkaSupervisorIOConfig; -import org.apache.hadoop.hive.druid.json.KafkaSupervisorReport; +import org.apache.hadoop.hive.druid.json.KafkaSupervisorReportPayload; import org.apache.hadoop.hive.druid.json.KafkaSupervisorSpec; import org.apache.hadoop.hive.druid.json.KafkaSupervisorTuningConfig; import org.apache.hadoop.hive.druid.security.KerberosHttpClient; @@ -356,7 +359,9 @@ private static KafkaSupervisorSpec createKafkaSupervisorSpec(Table table, String return new KafkaSupervisorSpec(dataSchema, new KafkaSupervisorTuningConfig( getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxRowsInMemory"), + getLongProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxBytesInMemory"), getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxRowsPerSegment"), + getLongProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxTotalRows"), getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "intermediatePersistPeriod"), null, // basePersistDirectory - use druid default, no need to be configured by user getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxPendingPersists"), @@ -365,13 +370,18 @@ private static KafkaSupervisorSpec createKafkaSupervisorSpec(Table table, String getBooleanProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "reportParseExceptions"), getLongProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "handoffConditionTimeout"), getBooleanProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "resetOffsetAutomatically"), + null, getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "workerThreads"), getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "chatThreads"), getLongProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "chatRetries"), getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "httpTimeout"), getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "shutdownTimeout"), - getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "offsetFetchPeriod")), - new KafkaSupervisorIOConfig(kafkaTopic, // Mandatory Property + getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "offsetFetchPeriod"), + getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "intermediateHandoffPeriod"), + getBooleanProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "logParseExceptions"), + getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxParseExceptions"), + getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxSavedParseExceptions")), + new KafkaSupervisorIOConfig(kafkaTopic, // Mandatory Property getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "replicas"), getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "taskCount"), getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "taskDuration"), @@ -535,7 +545,7 @@ private KafkaSupervisorSpec fetchKafkaIngestionSpec(Table table) { * @return kafka supervisor report or null when druid overlord is unreachable. */ @Nullable - private KafkaSupervisorReport fetchKafkaSupervisorReport(Table table) { + private SupervisorReport fetchKafkaSupervisorReport(Table table) { final String overlordAddress = Preconditions.checkNotNull(HiveConf .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), "Druid Overlord Address is null"); @@ -557,7 +567,10 @@ private KafkaSupervisorReport fetchKafkaSupervisorReport(Table table) { ); if (response.getStatus().equals(HttpResponseStatus.OK)) { return DruidStorageHandlerUtils.JSON_MAPPER - .readValue(response.getContent(), KafkaSupervisorReport.class); + .readValue(response.getContent(), + new TypeReference>() { + } + ); // Druid Returns 400 Bad Request when not found. } else if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) || response.getStatus().equals(HttpResponseStatus.BAD_REQUEST)) { LOG.info("No Kafka Supervisor found for datasource[%s]", dataSourceName); @@ -974,7 +987,8 @@ private SQLMetadataConnector buildConnector() { ); } else if (dbType.equals("postgresql")) { connector = new PostgreSQLConnector(storageConnectorConfigSupplier, - Suppliers.ofInstance(getDruidMetadataStorageTablesConfig()) + Suppliers.ofInstance(getDruidMetadataStorageTablesConfig()), + new PostgreSQLConnectorConfig() ); } else if (dbType.equals("derby")) { @@ -1126,7 +1140,7 @@ private int getMaxRetryCount() { @Override public StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaException { if(isKafkaStreamingTable(table)){ - KafkaSupervisorReport kafkaSupervisorReport = fetchKafkaSupervisorReport(table); + SupervisorReport kafkaSupervisorReport = fetchKafkaSupervisorReport(table); if(kafkaSupervisorReport == null){ return DruidStorageHandlerInfo.UNREACHABLE; } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java index f0e175057f5bc8a4585d8a40afe58756885813d3..5cdba43cf487de5f01e46ec8651e2f7e27d8e08d 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java @@ -17,9 +17,8 @@ */ package org.apache.hadoop.hive.druid; -import io.druid.java.util.common.StringUtils; - -import org.apache.hadoop.hive.druid.json.KafkaSupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.hadoop.hive.druid.json.KafkaSupervisorReportPayload; import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo; /** @@ -35,9 +34,9 @@ public String formatAsText() { } }; - private final KafkaSupervisorReport kafkaSupervisorReport; + private final SupervisorReport kafkaSupervisorReport; - DruidStorageHandlerInfo(KafkaSupervisorReport kafkaSupervisorReport) { + DruidStorageHandlerInfo(SupervisorReport kafkaSupervisorReport) { this.kafkaSupervisorReport = kafkaSupervisorReport; } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java index b9eb367f0fb4bb1320ff2a85eec034bef0df0bfd..26321e1560d5a284766eb527ea5eab7db0fbbbf1 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java @@ -31,61 +31,62 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.io.CharStreams; -import io.druid.data.input.impl.DimensionSchema; -import io.druid.data.input.impl.StringDimensionSchema; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.JodaUtils; -import io.druid.java.util.common.MapUtils; -import io.druid.java.util.common.Pair; -import io.druid.java.util.common.granularity.Granularity; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.core.NoopEmitter; -import io.druid.java.util.emitter.service.ServiceEmitter; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.Request; -import io.druid.java.util.http.client.response.FullResponseHandler; -import io.druid.java.util.http.client.response.FullResponseHolder; -import io.druid.java.util.http.client.response.InputStreamResponseHandler; -import io.druid.math.expr.ExprMacroTable; -import io.druid.metadata.MetadataStorageTablesConfig; -import io.druid.metadata.SQLMetadataConnector; -import io.druid.metadata.storage.mysql.MySQLConnector; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.DoubleSumAggregatorFactory; -import io.druid.query.aggregation.FloatSumAggregatorFactory; -import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.expression.LikeExprMacro; -import io.druid.query.expression.RegexpExtractExprMacro; -import io.druid.query.expression.TimestampCeilExprMacro; -import io.druid.query.expression.TimestampExtractExprMacro; -import io.druid.query.expression.TimestampFloorExprMacro; -import io.druid.query.expression.TimestampFormatExprMacro; -import io.druid.query.expression.TimestampParseExprMacro; -import io.druid.query.expression.TimestampShiftExprMacro; -import io.druid.query.expression.TrimExprMacro; -import io.druid.query.scan.ScanQuery; -import io.druid.query.select.SelectQueryConfig; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.segment.IndexIO; -import io.druid.segment.IndexMergerV9; -import io.druid.segment.IndexSpec; -import io.druid.segment.data.ConciseBitmapSerdeFactory; -import io.druid.segment.data.RoaringBitmapSerdeFactory; -import io.druid.segment.indexing.granularity.GranularitySpec; -import io.druid.segment.indexing.granularity.UniformGranularitySpec; -import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; -import io.druid.storage.hdfs.HdfsDataSegmentPusher; -import io.druid.storage.hdfs.HdfsDataSegmentPusherConfig; -import io.druid.timeline.DataSegment; -import io.druid.timeline.TimelineObjectHolder; -import io.druid.timeline.VersionedIntervalTimeline; -import io.druid.timeline.partition.LinearShardSpec; -import io.druid.timeline.partition.NoneShardSpec; -import io.druid.timeline.partition.NumberedShardSpec; -import io.druid.timeline.partition.PartitionChunk; -import io.druid.timeline.partition.ShardSpec; + +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.MapUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.FullResponseHandler; +import org.apache.druid.java.util.http.client.response.FullResponseHolder; +import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.storage.mysql.MySQLConnector; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.expression.LikeExprMacro; +import org.apache.druid.query.expression.RegexpExtractExprMacro; +import org.apache.druid.query.expression.TimestampCeilExprMacro; +import org.apache.druid.query.expression.TimestampExtractExprMacro; +import org.apache.druid.query.expression.TimestampFloorExprMacro; +import org.apache.druid.query.expression.TimestampFormatExprMacro; +import org.apache.druid.query.expression.TimestampParseExprMacro; +import org.apache.druid.query.expression.TimestampShiftExprMacro; +import org.apache.druid.query.expression.TrimExprMacro; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.select.SelectQueryConfig; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier; +import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.storage.hdfs.HdfsDataSegmentPusher; +import org.apache.druid.storage.hdfs.HdfsDataSegmentPusherConfig; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.PartitionChunk; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -243,7 +244,7 @@ * * @throws IOException */ - public static Request createSmileRequest(String address, io.druid.query.Query query) + public static Request createSmileRequest(String address, org.apache.druid.query.Query query) throws IOException { return new Request(HttpMethod.POST, new URL(String.format("%s/druid/v2/", "http://" + address))) .setContent(SMILE_MAPPER.writeValueAsBytes(query)) diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java index ecb43606233a286da23ae7d2f7466fa69220ea20..df679f1fac0ceb89e1139b7fd3dc83d47fcd2c77 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java @@ -17,19 +17,19 @@ */ package org.apache.hadoop.hive.druid.io; -import io.druid.data.input.impl.DimensionSchema; -import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.MapInputRowParser; -import io.druid.data.input.impl.TimeAndDimsParseSpec; -import io.druid.data.input.impl.TimestampSpec; -import io.druid.java.util.common.Pair; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.IndexSpec; -import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; -import io.druid.segment.indexing.granularity.GranularitySpec; -import io.druid.segment.realtime.plumber.CustomVersioningPolicy; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.plumber.CustomVersioningPolicy; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.FileSystem; @@ -144,6 +144,7 @@ IndexSpec indexSpec = DruidStorageHandlerUtils.getIndexSpec(jc); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(maxRowInMemory, + null, null, null, new File(basePersistDirectory, dataSource), @@ -156,8 +157,9 @@ 0, 0, true, - null, 0L, + null, + null, null ); diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java index f5009a27766f2b70df75d8b98a86a83b0f4462d9..102a3bde2121551c1358fdd2151cce53b9bd6692 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java @@ -21,15 +21,15 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonMappingException; import com.google.common.collect.Lists; -import io.druid.java.util.http.client.Request; -import io.druid.query.BaseQuery; -import io.druid.query.LocatedSegmentDescriptor; -import io.druid.query.Query; -import io.druid.query.SegmentDescriptor; -import io.druid.query.scan.ScanQuery; -import io.druid.query.select.PagingSpec; -import io.druid.query.select.SelectQuery; -import io.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.LocatedSegmentDescriptor; +import org.apache.druid.query.Query; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.select.PagingSpec; +import org.apache.druid.query.select.SelectQuery; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.commons.lang3.StringEscapeUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java index 400262a10731ae65ad81ffd00770ee05c828d9a4..83ea7214b09cdada67570daaca733cda4039a8cb 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java @@ -26,23 +26,23 @@ import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import io.druid.data.input.Committer; -import io.druid.data.input.InputRow; -import io.druid.data.input.MapBasedInputRow; -import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.granularity.Granularity; -import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; -import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.realtime.FireDepartmentMetrics; -import io.druid.segment.realtime.appenderator.Appenderator; -import io.druid.segment.realtime.appenderator.Appenderators; -import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import io.druid.segment.realtime.appenderator.SegmentNotWritableException; -import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; -import io.druid.segment.realtime.plumber.Committers; -import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier; +import org.apache.druid.segment.realtime.appenderator.SegmentNotWritableException; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import org.apache.druid.segment.realtime.plumber.Committers; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java index 425a5bbfc5ca107495a8bacf0cd3eccf519192ee..d2287fb902413490366914871071440d45dae088 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hive.druid.json; -import io.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.StringUtils; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java deleted file mode 100644 index 5a6756ecbf16e3cd56eeb14d8ffc161d6f53b479..0000000000000000000000000000000000000000 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java +++ /dev/null @@ -1,231 +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.hadoop.hive.druid.json; - -import io.druid.guice.annotations.Json; -import io.druid.indexing.overlord.supervisor.SupervisorReport; -import io.druid.java.util.common.IAE; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Lists; - -import org.joda.time.DateTime; - -import java.util.List; -import java.util.Map; - -import javax.annotation.Nullable; - -/** - * This class is copied from druid source code - * in order to avoid adding additional dependencies on druid-indexing-service. - */ -public class KafkaSupervisorReport extends SupervisorReport -{ - public static class KafkaSupervisorReportPayload - { - private final String dataSource; - private final String topic; - private final Integer partitions; - private final Integer replicas; - private final Long durationSeconds; - private final List activeTasks; - private final List publishingTasks; - private final Map latestOffsets; - private final Map minimumLag; - private final Long aggregateLag; - private final DateTime offsetsLastUpdated; - - @JsonCreator - public KafkaSupervisorReportPayload( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("topic") String topic, - @JsonProperty("partitions") Integer partitions, - @JsonProperty("replicas") Integer replicas, - @JsonProperty("durationSeconds") Long durationSeconds, - @Nullable @JsonProperty("latestOffsets") Map latestOffsets, - @Nullable @JsonProperty("minimumLag") Map minimumLag, - @Nullable @JsonProperty("aggregateLag") Long aggregateLag, - @Nullable @JsonProperty("offsetsLastUpdated") DateTime offsetsLastUpdated - ) - { - this.dataSource = dataSource; - this.topic = topic; - this.partitions = partitions; - this.replicas = replicas; - this.durationSeconds = durationSeconds; - this.activeTasks = Lists.newArrayList(); - this.publishingTasks = Lists.newArrayList(); - this.latestOffsets = latestOffsets; - this.minimumLag = minimumLag; - this.aggregateLag = aggregateLag; - this.offsetsLastUpdated = offsetsLastUpdated; - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public String getTopic() - { - return topic; - } - - @JsonProperty - public Integer getPartitions() - { - return partitions; - } - - @JsonProperty - public Integer getReplicas() - { - return replicas; - } - - @JsonProperty - public Long getDurationSeconds() - { - return durationSeconds; - } - - @JsonProperty - public List getActiveTasks() - { - return activeTasks; - } - - @JsonProperty - public List getPublishingTasks() - { - return publishingTasks; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLatestOffsets() - { - return latestOffsets; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getMinimumLag() - { - return minimumLag; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getAggregateLag() - { - return aggregateLag; - } - - @JsonProperty - public DateTime getOffsetsLastUpdated() - { - return offsetsLastUpdated; - } - - @Override - public String toString() - { - return "{" + - "dataSource='" + dataSource + '\'' + - ", topic='" + topic + '\'' + - ", partitions=" + partitions + - ", replicas=" + replicas + - ", durationSeconds=" + durationSeconds + - ", active=" + activeTasks + - ", publishing=" + publishingTasks + - (latestOffsets != null ? ", latestOffsets=" + latestOffsets : "") + - (minimumLag != null ? ", minimumLag=" + minimumLag : "") + - (aggregateLag != null ? ", aggregateLag=" + aggregateLag : "") + - (offsetsLastUpdated != null ? ", offsetsLastUpdated=" + offsetsLastUpdated : "") + - '}'; - } - } - - private final KafkaSupervisorReportPayload payload; - - @JsonCreator - public KafkaSupervisorReport(@JsonProperty("id") String id, - @JsonProperty("generationTime")DateTime generationTime, - @JsonProperty("payload") KafkaSupervisorReportPayload payload){ - super(id, generationTime); - this.payload = payload; - } - - public KafkaSupervisorReport( - String dataSource, - DateTime generationTime, - String topic, - Integer partitions, - Integer replicas, - Long durationSeconds, - @Nullable Map latestOffsets, - @Nullable Map minimumLag, - @Nullable Long aggregateLag, - @Nullable DateTime offsetsLastUpdated - ) { - this(dataSource, generationTime, new KafkaSupervisorReportPayload( - dataSource, - topic, - partitions, - replicas, - durationSeconds, - latestOffsets, - minimumLag, - aggregateLag, - offsetsLastUpdated - )); - } - - @Override - public KafkaSupervisorReportPayload getPayload() - { - return payload; - } - - public void addTask(TaskReportData data) - { - if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { - payload.activeTasks.add(data); - } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { - payload.publishingTasks.add(data); - } else { - throw new IAE("Unknown task type [%s]", data.getType().name()); - } - } - - @Override - public String toString() - { - return "{" + - "id='" + getId() + '\'' + - ", generationTime=" + getGenerationTime() + - ", payload=" + payload + - '}'; - } -} diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReportPayload.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReportPayload.java new file mode 100644 index 0000000000000000000000000000000000000000..0f605ee11685c3e669ca9bb0aeebf704fee1ee0d --- /dev/null +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReportPayload.java @@ -0,0 +1,178 @@ +/* + * 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.hadoop.hive.druid.json; + +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.java.util.common.IAE; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; + +import org.joda.time.DateTime; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * This class is copied from druid source code + * in order to avoid adding additional dependencies on druid-indexing-service. + */ +public class KafkaSupervisorReportPayload +{ + private final String dataSource; + private final String topic; + private final int partitions; + private final int replicas; + private final long durationSeconds; + private final List activeTasks; + private final List publishingTasks; + private final Map latestOffsets; + private final Map minimumLag; + private final Long aggregateLag; + private final DateTime offsetsLastUpdated; + + public KafkaSupervisorReportPayload( + String dataSource, + String topic, + int partitions, + int replicas, + long durationSeconds, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, + @Nullable Long aggregateLag, + @Nullable DateTime offsetsLastUpdated + ) + { + this.dataSource = dataSource; + this.topic = topic; + this.partitions = partitions; + this.replicas = replicas; + this.durationSeconds = durationSeconds; + this.activeTasks = new ArrayList<>(); + this.publishingTasks = new ArrayList<>(); + this.latestOffsets = latestOffsets; + this.minimumLag = minimumLag; + this.aggregateLag = aggregateLag; + this.offsetsLastUpdated = offsetsLastUpdated; + } + + public void addTask(TaskReportData data) + { + if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { + activeTasks.add(data); + } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { + publishingTasks.add(data); + } else { + throw new IAE("Unknown task type [%s]", data.getType().name()); + } + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public String getTopic() + { + return topic; + } + + @JsonProperty + public int getPartitions() + { + return partitions; + } + + @JsonProperty + public int getReplicas() + { + return replicas; + } + + @JsonProperty + public long getDurationSeconds() + { + return durationSeconds; + } + + @JsonProperty + public List getActiveTasks() + { + return activeTasks; + } + + @JsonProperty + public List getPublishingTasks() + { + return publishingTasks; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getLatestOffsets() + { + return latestOffsets; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getMinimumLag() + { + return minimumLag; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getAggregateLag() + { + return aggregateLag; + } + + @JsonProperty + public DateTime getOffsetsLastUpdated() + { + return offsetsLastUpdated; + } + + @Override + public String toString() + { + return "{" + + "dataSource='" + dataSource + '\'' + + ", topic='" + topic + '\'' + + ", partitions=" + partitions + + ", replicas=" + replicas + + ", durationSeconds=" + durationSeconds + + ", active=" + activeTasks + + ", publishing=" + publishingTasks + + (latestOffsets != null ? ", latestOffsets=" + latestOffsets : "") + + (minimumLag != null ? ", minimumLag=" + minimumLag : "") + + (aggregateLag != null ? ", aggregateLag=" + aggregateLag : "") + + (offsetsLastUpdated != null ? ", offsetsLastUpdated=" + offsetsLastUpdated : "") + + '}'; + } +} + diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorSpec.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorSpec.java index 081bc279b8c7f27a26251ae3b6974182fdd3304f..c2e17c9e8020b3dd3c8ac54ec01a70b1808230d3 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorSpec.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorSpec.java @@ -18,14 +18,24 @@ package org.apache.hadoop.hive.druid.json; -import io.druid.segment.indexing.DataSchema; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.segment.indexing.DataSchema; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Preconditions; +import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; +import java.util.List; import java.util.Map; /** @@ -40,7 +50,7 @@ name = "kafka", value = KafkaSupervisorSpec.class )}) -public class KafkaSupervisorSpec +public class KafkaSupervisorSpec implements SupervisorSpec { private final DataSchema dataSchema; private final KafkaSupervisorTuningConfig tuningConfig; @@ -49,33 +59,40 @@ @JsonCreator public KafkaSupervisorSpec( - @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("tuningConfig") KafkaSupervisorTuningConfig tuningConfig, - @JsonProperty("ioConfig") KafkaSupervisorIOConfig ioConfig, - @JsonProperty("context") Map context + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KafkaSupervisorTuningConfig tuningConfig, + @JsonProperty("ioConfig") KafkaSupervisorIOConfig ioConfig, + @JsonProperty("context") Map context ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.tuningConfig = tuningConfig != null - ? tuningConfig - : new KafkaSupervisorTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + ? tuningConfig + : new KafkaSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.context = context; } @@ -104,14 +121,32 @@ public KafkaSupervisorIOConfig getIoConfig() return context; } + @Override + public String getId() + { + return dataSchema.getDataSource(); + } + + @Override + public Supervisor createSupervisor() + { + return null; + } + + @Override + public List getDataSources() + { + return ImmutableList.of(getDataSchema().getDataSource()); + } + @Override public String toString() { return "KafkaSupervisorSpec{" + - "dataSchema=" + dataSchema + - ", tuningConfig=" + tuningConfig + - ", ioConfig=" + ioConfig + - '}'; + "dataSchema=" + dataSchema + + ", tuningConfig=" + tuningConfig + + ", ioConfig=" + ioConfig + + '}'; } @Override diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java index a918df475d5ac509aa4b5317404be54f6ebab702..adc9ea596002aef715ef932a010d8ff90ea191c0 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java @@ -18,15 +18,18 @@ package org.apache.hadoop.hive.druid.json; -import io.druid.segment.IndexSpec; +import org.apache.druid.segment.IndexSpec; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.segment.indexing.TuningConfigs; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; /** @@ -51,38 +54,51 @@ private final Duration offsetFetchPeriod; public KafkaSupervisorTuningConfig( - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, - @JsonProperty("basePersistDirectory") File basePersistDirectory, - @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("indexSpec") IndexSpec indexSpec, - // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. - @JsonProperty("buildV9Directly") Boolean buildV9Directly, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, // for backward compatibility - @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, - @JsonProperty("workerThreads") Integer workerThreads, - @JsonProperty("chatThreads") Integer chatThreads, - @JsonProperty("chatRetries") Long chatRetries, - @JsonProperty("httpTimeout") Period httpTimeout, - @JsonProperty("shutdownTimeout") Period shutdownTimeout, - @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") Long maxTotalRows, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("workerThreads") Integer workerThreads, + @JsonProperty("chatThreads") Integer chatThreads, + @JsonProperty("chatRetries") Long chatRetries, + @JsonProperty("httpTimeout") Period httpTimeout, + @JsonProperty("shutdownTimeout") Period shutdownTimeout, + @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, + @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { super( - maxRowsInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - basePersistDirectory, - maxPendingPersists, - indexSpec, - true, - reportParseExceptions, - // Supervised kafka tasks should respect KafkaSupervisorIOConfig.completionTimeout instead of - // handoffConditionTimeout - handoffConditionTimeout, - resetOffsetAutomatically + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); this.workerThreads = workerThreads; @@ -133,68 +149,33 @@ public Duration getOffsetFetchPeriod() public String toString() { return "KafkaSupervisorTuningConfig{" + - "maxRowsInMemory=" + getMaxRowsInMemory() + - ", maxRowsPerSegment=" + getMaxRowsPerSegment() + - ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + - ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + getMaxPendingPersists() + - ", indexSpec=" + getIndexSpec() + - ", reportParseExceptions=" + isReportParseExceptions() + - ", handoffConditionTimeout=" + getHandoffConditionTimeout() + - ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + - ", workerThreads=" + workerThreads + - ", chatThreads=" + chatThreads + - ", chatRetries=" + chatRetries + - ", httpTimeout=" + httpTimeout + - ", shutdownTimeout=" + shutdownTimeout + - ", offsetFetchPeriod=" + offsetFetchPeriod + - '}'; + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxTotalRows=" + getMaxTotalRows() + + ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", basePersistDirectory=" + getBasePersistDirectory() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", workerThreads=" + workerThreads + + ", chatThreads=" + chatThreads + + ", chatRetries=" + chatRetries + + ", httpTimeout=" + httpTimeout + + ", shutdownTimeout=" + shutdownTimeout + + ", offsetFetchPeriod=" + offsetFetchPeriod + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + + '}'; } private static Duration defaultDuration(final Period period, final String theDefault) { return (period == null ? new Period(theDefault) : period).toStandardDuration(); } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - if (!super.equals(o)) - return false; - - KafkaSupervisorTuningConfig that = (KafkaSupervisorTuningConfig) o; - - if (workerThreads != null ? - !workerThreads.equals(that.workerThreads) : - that.workerThreads != null) - return false; - if (chatThreads != null ? !chatThreads.equals(that.chatThreads) : that.chatThreads != null) - return false; - if (chatRetries != null ? !chatRetries.equals(that.chatRetries) : that.chatRetries != null) - return false; - if (httpTimeout != null ? !httpTimeout.equals(that.httpTimeout) : that.httpTimeout != null) - return false; - if (shutdownTimeout != null ? - !shutdownTimeout.equals(that.shutdownTimeout) : - that.shutdownTimeout != null) - return false; - return offsetFetchPeriod != null ? - offsetFetchPeriod.equals(that.offsetFetchPeriod) : - that.offsetFetchPeriod == null; - } - - @Override - public int hashCode() { - int result = super.hashCode(); - result = 31 * result + (workerThreads != null ? workerThreads.hashCode() : 0); - result = 31 * result + (chatThreads != null ? chatThreads.hashCode() : 0); - result = 31 * result + (chatRetries != null ? chatRetries.hashCode() : 0); - result = 31 * result + (httpTimeout != null ? httpTimeout.hashCode() : 0); - result = 31 * result + (shutdownTimeout != null ? shutdownTimeout.hashCode() : 0); - result = 31 * result + (offsetFetchPeriod != null ? offsetFetchPeriod.hashCode() : 0); - return result; - } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java index 1ec8b5c2381151ddca292b2212e436ded257e438..21808ed57a8b22b6763d94faa7b535dca62a0958 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java @@ -18,52 +18,72 @@ package org.apache.hadoop.hive.druid.json; -import io.druid.segment.IndexSpec; -import io.druid.segment.indexing.RealtimeTuningConfig; -import io.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; import javax.annotation.Nullable; import java.io.File; +import java.util.Objects; /** * This class is copied from druid source code * in order to avoid adding additional dependencies on druid-indexing-service. */ -public class KafkaTuningConfig implements AppenderatorConfig +public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig { private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; private final int maxRowsInMemory; + private final long maxBytesInMemory; private final int maxRowsPerSegment; + @Nullable + private final Long maxTotalRows; private final Period intermediatePersistPeriod; private final File basePersistDirectory; + @Deprecated private final int maxPendingPersists; private final IndexSpec indexSpec; private final boolean reportParseExceptions; @Deprecated private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + private final Period intermediateHandoffPeriod; + + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; @JsonCreator public KafkaTuningConfig( - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, - @JsonProperty("basePersistDirectory") File basePersistDirectory, - @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("indexSpec") IndexSpec indexSpec, - // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. - @JsonProperty("buildV9Directly") Boolean buildV9Directly, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, + @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. + @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, + @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { // Cannot be a static because default basePersistDirectory is unique per-instance @@ -71,36 +91,62 @@ public KafkaTuningConfig( this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + // initializing this to 0, it will be lazily initialized to a value + // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; + this.maxTotalRows = maxTotalRows; this.intermediatePersistPeriod = intermediatePersistPeriod == null - ? defaults.getIntermediatePersistPeriod() - : intermediatePersistPeriod; + ? defaults.getIntermediatePersistPeriod() + : intermediatePersistPeriod; this.basePersistDirectory = defaults.getBasePersistDirectory(); - this.maxPendingPersists = maxPendingPersists == null ? defaults.getMaxPendingPersists() : maxPendingPersists; + this.maxPendingPersists = 0; this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; this.reportParseExceptions = reportParseExceptions == null - ? defaults.isReportParseExceptions() - : reportParseExceptions; + ? defaults.isReportParseExceptions() + : reportParseExceptions; this.handoffConditionTimeout = handoffConditionTimeout == null - ? defaults.getHandoffConditionTimeout() - : handoffConditionTimeout; + ? defaults.getHandoffConditionTimeout() + : handoffConditionTimeout; this.resetOffsetAutomatically = resetOffsetAutomatically == null - ? DEFAULT_RESET_OFFSET_AUTOMATICALLY - : resetOffsetAutomatically; + ? DEFAULT_RESET_OFFSET_AUTOMATICALLY + : resetOffsetAutomatically; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.intermediateHandoffPeriod = intermediateHandoffPeriod == null + ? new Period().withDays(Integer.MAX_VALUE) + : intermediateHandoffPeriod; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; } public static KafkaTuningConfig copyOf(KafkaTuningConfig config) { return new KafkaTuningConfig( - config.maxRowsInMemory, - config.maxRowsPerSegment, - config.intermediatePersistPeriod, - config.basePersistDirectory, - config.maxPendingPersists, - config.indexSpec, - true, - config.reportParseExceptions, - config.handoffConditionTimeout, - config.resetOffsetAutomatically + config.maxRowsInMemory, + config.maxBytesInMemory, + config.maxRowsPerSegment, + config.maxTotalRows, + config.intermediatePersistPeriod, + config.basePersistDirectory, + config.maxPendingPersists, + config.indexSpec, + true, + config.reportParseExceptions, + config.handoffConditionTimeout, + config.resetOffsetAutomatically, + config.segmentWriteOutMediumFactory, + config.intermediateHandoffPeriod, + config.logParseExceptions, + config.maxParseExceptions, + config.maxSavedParseExceptions ); } @@ -111,12 +157,29 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @Override + @JsonProperty + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @Override @JsonProperty public int getMaxRowsPerSegment() { return maxRowsPerSegment; } + + @JsonProperty + @Override + @Nullable + public Long getMaxTotalRows() + { + return maxTotalRows; + } + @Override @JsonProperty public Period getIntermediatePersistPeriod() @@ -131,12 +194,9 @@ public File getBasePersistDirectory() return basePersistDirectory; } - @Nullable @Override public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { - return null; - } - @Override @JsonProperty + @Deprecated public int getMaxPendingPersists() { return maxPendingPersists; @@ -166,7 +226,6 @@ public boolean isReportParseExceptions() return reportParseExceptions; } - @Deprecated @JsonProperty public long getHandoffConditionTimeout() { @@ -179,19 +238,58 @@ public boolean isResetOffsetAutomatically() return resetOffsetAutomatically; } + @Override + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @JsonProperty + public Period getIntermediateHandoffPeriod() + { + return intermediateHandoffPeriod; + } + + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + public KafkaTuningConfig withBasePersistDirectory(File dir) { return new KafkaTuningConfig( - maxRowsInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - dir, - maxPendingPersists, - indexSpec, - true, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + dir, + maxPendingPersists, + indexSpec, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions ); } @@ -204,69 +302,68 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - KafkaTuningConfig that = (KafkaTuningConfig) o; - - if (maxRowsInMemory != that.maxRowsInMemory) { - return false; - } - if (maxRowsPerSegment != that.maxRowsPerSegment) { - return false; - } - if (maxPendingPersists != that.maxPendingPersists) { - return false; - } - if (reportParseExceptions != that.reportParseExceptions) { - return false; - } - if (handoffConditionTimeout != that.handoffConditionTimeout) { - return false; - } - if (resetOffsetAutomatically != that.resetOffsetAutomatically) { - return false; - } - if (intermediatePersistPeriod != null - ? !intermediatePersistPeriod.equals(that.intermediatePersistPeriod) - : that.intermediatePersistPeriod != null) { - return false; - } - if (basePersistDirectory != null - ? !basePersistDirectory.equals(that.basePersistDirectory) - : that.basePersistDirectory != null) { - return false; - } - return indexSpec != null ? indexSpec.equals(that.indexSpec) : that.indexSpec == null; - + return maxRowsInMemory == that.maxRowsInMemory && + maxRowsPerSegment == that.maxRowsPerSegment && + maxBytesInMemory == that.maxBytesInMemory && + Objects.equals(maxTotalRows, that.maxTotalRows) && + maxPendingPersists == that.maxPendingPersists && + reportParseExceptions == that.reportParseExceptions && + handoffConditionTimeout == that.handoffConditionTimeout && + resetOffsetAutomatically == that.resetOffsetAutomatically && + Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && + Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod) && + logParseExceptions == that.logParseExceptions && + maxParseExceptions == that.maxParseExceptions && + maxSavedParseExceptions == that.maxSavedParseExceptions; } @Override public int hashCode() { - int result = maxRowsInMemory; - result = 31 * result + maxRowsPerSegment; - result = 31 * result + (intermediatePersistPeriod != null ? intermediatePersistPeriod.hashCode() : 0); - result = 31 * result + (basePersistDirectory != null ? basePersistDirectory.hashCode() : 0); - result = 31 * result + maxPendingPersists; - result = 31 * result + (indexSpec != null ? indexSpec.hashCode() : 0); - result = 31 * result + (reportParseExceptions ? 1 : 0); - result = 31 * result + (int) (handoffConditionTimeout ^ (handoffConditionTimeout >>> 32)); - result = 31 * result + (resetOffsetAutomatically ? 1 : 0); - return result; + return Objects.hash( + maxRowsInMemory, + maxRowsPerSegment, + maxBytesInMemory, + maxTotalRows, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); } @Override public String toString() { return "KafkaTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxRowsPerSegment=" + maxRowsPerSegment + - ", intermediatePersistPeriod=" + intermediatePersistPeriod + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", indexSpec=" + indexSpec + - ", reportParseExceptions=" + reportParseExceptions + - ", handoffConditionTimeout=" + handoffConditionTimeout + - ", resetOffsetAutomatically=" + resetOffsetAutomatically + - '}'; + "maxRowsInMemory=" + maxRowsInMemory + + ", maxRowsPerSegment=" + maxRowsPerSegment + + ", maxTotalRows=" + maxTotalRows + + ", maxBytesInMemory=" + maxBytesInMemory + + ", intermediatePersistPeriod=" + intermediatePersistPeriod + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", indexSpec=" + indexSpec + + ", reportParseExceptions=" + reportParseExceptions + + ", handoffConditionTimeout=" + handoffConditionTimeout + + ", resetOffsetAutomatically=" + resetOffsetAutomatically + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + ", intermediateHandoffPeriod=" + intermediateHandoffPeriod + + ", logParseExceptions=" + logParseExceptions + + ", maxParseExceptions=" + maxParseExceptions + + ", maxSavedParseExceptions=" + maxSavedParseExceptions + + '}'; } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java index 4644df1fcc03e39fd17f8d276b7aae9e412a5279..8b520bb756c92c9aad639409dab4e02497f354f7 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java @@ -21,10 +21,10 @@ import com.google.common.base.Throwables; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; -import io.druid.java.util.http.client.AbstractHttpClient; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.Request; -import io.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.java.util.http.client.AbstractHttpClient; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.hadoop.security.UserGroupInformation; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.joda.time.Duration; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java index 221081ae7d93dbb0409d00cdb5856350b21fb61e..2b963a59c0cb9238db8e811c61093683edd8088b 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java @@ -19,8 +19,8 @@ package org.apache.hadoop.hive.druid.security; import com.google.common.collect.Maps; -import io.druid.java.util.http.client.response.ClientResponse; -import io.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.java.util.http.client.response.ClientResponse; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java index ff169e88c8836eef8ec63a18541841297f93b60b..4057739cd30219b40dfa6d46386ccdc359486537 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hive.druid.security; -import io.druid.java.util.http.client.response.ClientResponse; -import io.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.java.util.http.client.response.ClientResponse; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpResponse; import org.jboss.netty.handler.codec.http.HttpResponseStatus; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java index 611cad38ca91accfc3185a7f0b8c4b2fe662211f..c49a6e3358bca6d89b586f84407ece5eb1b0f076 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java @@ -20,10 +20,10 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.java.util.http.client.HttpClient; -import io.druid.data.input.MapBasedRow; -import io.druid.data.input.Row; -import io.druid.query.groupby.GroupByQuery; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.groupby.GroupByQuery; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.io.NullWritable; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java index 8c10261a2ee293d0a4bdc3342199ee06f069106b..65585d016c2160b84866f81d6ccb2ef36a4c2fda 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java @@ -24,15 +24,15 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import io.druid.java.util.common.IAE; -import io.druid.java.util.common.RE; -import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.Request; -import io.druid.java.util.http.client.response.InputStreamResponseHandler; -import io.druid.query.BaseQuery; -import io.druid.query.Query; -import io.druid.query.QueryInterruptedException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.guava.CloseQuietly; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryInterruptedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.druid.DruidStorageHandler; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java index 6c6514fea0bafe6d32973ab4520ba1741e8bedfe..9cd40fa0819a5b6dc4ef96e6616c37fdd34d3b17 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hive.druid.serde; -import io.druid.query.scan.ScanQuery; -import io.druid.query.scan.ScanResultValue; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanResultValue; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.io.NullWritable; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java index 1ccca5f1172c3cf798c7f48aa8d1fbff9a607280..3d9c5a8b4393ac5013d056321a0d2688f4f93953 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java @@ -27,10 +27,10 @@ import com.fasterxml.jackson.core.type.TypeReference; -import io.druid.query.Result; -import io.druid.query.select.EventHolder; -import io.druid.query.select.SelectQuery; -import io.druid.query.select.SelectResultValue; +import org.apache.druid.query.Result; +import org.apache.druid.query.select.EventHolder; +import org.apache.druid.query.select.SelectQuery; +import org.apache.druid.query.select.SelectResultValue; /** * Record reader for results for Druid SelectQuery. diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java index 946a0753ee204287abfa7820bea5b30d8726e109..a3095f6e1d6c03f53cc94f0dd21eab3f1c25b19f 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java @@ -21,11 +21,11 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import io.druid.query.Druids; -import io.druid.query.Druids.SegmentMetadataQueryBuilder; -import io.druid.query.metadata.metadata.ColumnAnalysis; -import io.druid.query.metadata.metadata.SegmentAnalysis; -import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.Druids; +import org.apache.druid.query.Druids.SegmentMetadataQueryBuilder; +import org.apache.druid.query.metadata.metadata.ColumnAnalysis; +import org.apache.druid.query.metadata.metadata.SegmentAnalysis; +import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.type.Date; import org.apache.hadoop.hive.common.type.HiveChar; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java index fcda3efd34fe615f228a878008fadad5c1fea59e..3f82428ffe96f7e66eef183e42ff44d117eff52a 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java @@ -19,9 +19,9 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JavaType; -import io.druid.query.Result; -import io.druid.query.timeseries.TimeseriesQuery; -import io.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.query.Result; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.io.NullWritable; diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java index e24bfff495404153f2ee9d449d4cda675242ec5c..58398b3ef6247c1eaba1aa27892e0fced86fe10b 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java @@ -30,10 +30,10 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.Iterators; -import io.druid.query.Result; -import io.druid.query.topn.DimensionAndMetricValueExtractor; -import io.druid.query.topn.TopNQuery; -import io.druid.query.topn.TopNResultValue; +import org.apache.druid.query.Result; +import org.apache.druid.query.topn.DimensionAndMetricValueExtractor; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNResultValue; /** * Record reader for results for Druid TopNQuery. diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java index 627f0787cb562644789b18d77155cc98c279e87f..9b1b89ab55b6493c36b9700594af89d7678dfc81 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java @@ -20,10 +20,10 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import io.druid.metadata.MetadataStorageConnectorConfig; -import io.druid.metadata.MetadataStorageTablesConfig; -import io.druid.metadata.storage.derby.DerbyConnector; -import io.druid.metadata.storage.derby.DerbyMetadataStorage; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.storage.derby.DerbyConnector; +import org.apache.druid.metadata.storage.derby.DerbyMetadataStorage; import org.junit.Assert; import org.junit.rules.ExternalResource; diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java index ac3d2a51fe9953d4017e3beb0bbe7155389c9230..a99e5c38f3e50de9d9d5bade456e324bd3924a06 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java @@ -25,8 +25,8 @@ import com.fasterxml.jackson.core.type.TypeReference; -import io.druid.query.metadata.metadata.SegmentAnalysis; -import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.metadata.metadata.SegmentAnalysis; +import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; /** * Druid SerDe to be used in tests. diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java index f52e72176353bc8f9baa7c5882c93081b62e30c2..122d161de8f481ac681d9b0766f075fdd7a5a715 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java @@ -25,8 +25,8 @@ import com.fasterxml.jackson.core.type.TypeReference; -import io.druid.query.metadata.metadata.SegmentAnalysis; -import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.metadata.metadata.SegmentAnalysis; +import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; /** * Druid SerDe to be used in tests. diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java index 510330d5d08c69cda0c2219d5e4e3d86b1a65f40..8236b04f2c2b9e3e3d29dd678827ceb9a7718e1f 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java @@ -18,17 +18,17 @@ package org.apache.hadoop.hive.druid; -import io.druid.indexer.JobHelper; -import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler; -import io.druid.metadata.MetadataStorageTablesConfig; -import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.SegmentLoadingException; -import io.druid.storage.hdfs.HdfsDataSegmentPusher; -import io.druid.storage.hdfs.HdfsDataSegmentPusherConfig; -import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.LinearShardSpec; -import io.druid.timeline.partition.NoneShardSpec; -import io.druid.timeline.partition.ShardSpec; +import org.apache.druid.indexer.JobHelper; +import org.apache.druid.indexer.SQLMetadataStorageUpdaterJobHandler; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.storage.hdfs.HdfsDataSegmentPusher; +import org.apache.druid.storage.hdfs.HdfsDataSegmentPusherConfig; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java index bb43d512c41a8cd50b13c196f8656b9cfaeab873..a0423ffcd612393facf22b07c91c08b29bee4d16 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java @@ -28,7 +28,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.junit.Test; -import io.druid.query.Query; +import org.apache.druid.query.Query; import junit.framework.TestCase; public class TestHiveDruidQueryBasedInputFormat extends TestCase { diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java index 8b6c890e07416856c7a6d3298affafe8efe0c5c6..8dd842a41f90d897e9718eef7a69a4c24ca831c1 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java @@ -34,10 +34,10 @@ import java.util.Map.Entry; import java.util.Properties; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.response.HttpResponseHandler; -import io.druid.query.scan.ScanResultValue; -import io.druid.query.select.EventHolder; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.query.select.EventHolder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveChar; @@ -84,12 +84,12 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.SettableFuture; -import io.druid.data.input.Row; -import io.druid.query.Query; -import io.druid.query.Result; -import io.druid.query.select.SelectResultValue; -import io.druid.query.timeseries.TimeseriesResultValue; -import io.druid.query.topn.TopNResultValue; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.Query; +import org.apache.druid.query.Result; +import org.apache.druid.query.select.SelectResultValue; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.query.topn.TopNResultValue; /** * Basic tests for Druid SerDe. The examples are taken from Druid 0.9.1.1 diff --git a/druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java b/druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java index cb8fa3919bb63da713d1c40cf946b6d87f6a0e9c..815422965ad0818366329b03cff620f4dc87609e 100644 --- a/druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java +++ b/druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java @@ -23,34 +23,34 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import io.druid.data.input.Firehose; -import io.druid.data.input.InputRow; -import io.druid.data.input.impl.DimensionSchema; -import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.MapInputRowParser; -import io.druid.data.input.impl.StringDimensionSchema; -import io.druid.data.input.impl.TimeAndDimsParseSpec; -import io.druid.data.input.impl.TimestampSpec; -import io.druid.java.util.common.granularity.Granularities; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; -import io.druid.segment.IndexSpec; -import io.druid.segment.QueryableIndex; -import io.druid.segment.QueryableIndexStorageAdapter; -import io.druid.segment.data.RoaringBitmapSerdeFactory; -import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; -import io.druid.segment.indexing.granularity.UniformGranularitySpec; -import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.LocalDataSegmentPuller; -import io.druid.segment.loading.LocalDataSegmentPusher; -import io.druid.segment.loading.LocalDataSegmentPusherConfig; -import io.druid.segment.loading.SegmentLoadingException; -import io.druid.segment.realtime.firehose.IngestSegmentFirehose; -import io.druid.segment.realtime.firehose.WindowedStorageAdapter; -import io.druid.timeline.DataSegment; +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPuller; +import org.apache.druid.segment.loading.LocalDataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose; +import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.timeline.DataSegment; import org.apache.calcite.adapter.druid.DruidTable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -149,9 +149,9 @@ public void testWrite() throws IOException, SegmentLoadingException { ); IndexSpec indexSpec = new IndexSpec(new RoaringBitmapSerdeFactory(true), null, null, null); - RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig(null, null, null, + RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( null,null, null, null, temporaryFolder.newFolder(), null, null, null, null, indexSpec, null, 0, 0, null, null, - 0L, null + 0L, null, null ); LocalFileSystem localFileSystem = FileSystem.getLocal(config); DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher( diff --git a/itests/qtest-druid/pom.xml b/itests/qtest-druid/pom.xml index e566fcf4d706af969c4589f1bfa77c4cf80b04fc..22fa21611fac921dd06b0de5b57c93e921a34218 100644 --- a/itests/qtest-druid/pom.xml +++ b/itests/qtest-druid/pom.xml @@ -39,7 +39,7 @@ ../.. 4.0.0 1.19.3 - 9.3.19.v20170502 + 9.4.10.v20180503 10.11.1.1 16.0.1 4.1.0 @@ -47,7 +47,7 @@ - io.druid + org.apache.druid druid-server ${druid.version} @@ -66,7 +66,7 @@ - io.druid + org.apache.druid druid-services ${druid.version} @@ -90,7 +90,7 @@ ${druid.guava.version} - io.druid.extensions + org.apache.druid.extensions druid-hdfs-storage ${druid.version} @@ -105,7 +105,7 @@ - io.druid.extensions + org.apache.druid.extensions druid-kafka-indexing-service ${druid.version} @@ -230,7 +230,7 @@ false - io.druid.cli.Main + org.apache.druid.cli.Main diff --git a/itests/qtest-druid/src/main/java/org/apache/hive/druid/ForkingDruidNode.java b/itests/qtest-druid/src/main/java/org/apache/hive/druid/ForkingDruidNode.java index f81a0cae6bbc824022d69db834b4b73f1a954375..8480e47865cf15f711da0f218ad0d7d84da0df68 100644 --- a/itests/qtest-druid/src/main/java/org/apache/hive/druid/ForkingDruidNode.java +++ b/itests/qtest-druid/src/main/java/org/apache/hive/druid/ForkingDruidNode.java @@ -61,7 +61,7 @@ private final List allowedPrefixes = Lists.newArrayList( "com.metamx", "druid", - "io.druid", + "org.apache.druid", "java.io.tmpdir", "hadoop" ); @@ -114,7 +114,7 @@ public ForkingDruidNode(String nodeType, } this.properties .forEach((key, value) -> command.add(String.format("-D%s=%s", key, value))); - command.addAll(Lists.newArrayList("io.druid.cli.Main", "server", getNodeType())); + command.addAll(Lists.newArrayList("org.apache.druid.cli.Main", "server", getNodeType())); processBuilder.command(command); log.info("Creating forking druid node with " + String.join(" ", processBuilder.command())); } diff --git a/pom.xml b/pom.xml index 5008923d2af267b29ba83ef4ec128623eee76d41..360291d64ee1dcd943011886f918900c413d0f02 100644 --- a/pom.xml +++ b/pom.xml @@ -135,7 +135,7 @@ 1.2 1.7 3.2.2 - 1.9 + 1.16 1.1 2.4 2.6 @@ -145,7 +145,7 @@ 10.14.1.0 3.1.0 0.1.2 - 0.12.1 + 0.13.0-SNAPSHOT 1.2.0-3f79e055 19.0 2.4.11 diff --git a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out index 26ef4d2ad2ac6844c4f46aa171b71e44bb53fc5f..22bcafa31d03da684374a6a2f0f605a35c02e916 100644 --- a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out +++ b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out @@ -872,7 +872,7 @@ POSTHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_ POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6078 2.7308662809692383E7 +6078 2.7308662793799996E7 PREHOOK: query: select count(distinct cstring2), sum(2 * cdouble) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -881,7 +881,7 @@ POSTHOOK: query: select count(distinct cstring2), sum(2 * cdouble) FROM druid_ta POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6078 5.4617325619384766E7 +6078 5.461732558759999E7 PREHOOK: query: select count(DISTINCT cstring2) FROM druid_table_alltypesorc PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -899,7 +899,7 @@ POSTHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_ POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6078 2.7308662809692383E7 +6078 2.7308662793799996E7 PREHOOK: query: select count(distinct cstring2 || '_'|| cstring1), sum(cdouble), min(cint) FROM druid_table_alltypesorc PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -908,7 +908,7 @@ POSTHOOK: query: select count(distinct cstring2 || '_'|| cstring1), sum(cdouble) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6095 2.7308662809692383E7 -1073279343 +6095 2.7308662793799996E7 -1073279343 PREHOOK: query: explain select unix_timestamp(from_unixtime(1396681200)) from druid_table_alltypesorc limit 1 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -1270,7 +1270,7 @@ GROUP BY CAST(TRUNC(CAST(`druid_table_alias`.`__time` AS TIMESTAMP),'MM') AS DAT POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -3.832948323764436E14 1969-12-01 +3.83294831503382E14 1969-12-01 PREHOOK: query: explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1, DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc order by date_1, date_2 limit 3 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -1657,7 +1657,7 @@ POSTHOOK: query: SELECT TO_DATE(date1), TO_DATE(datetime1) FROM druid_table_n1 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_n1 POSTHOOK: Output: hdfs://### HDFS PATH ### -1970-01-01 1970-01-01 +NULL NULL 2004-04-04 2004-04-04 2004-04-09 2004-04-09 2004-04-11 2004-04-11 diff --git a/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out b/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out index 0106ffabae6620ba23fc6a533db894a3a574e82c..69c3436bd7606525016ba82582afaf928764fa6b 100644 --- a/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out +++ b/ql/src/test/results/clientpositive/druid/druidmini_mv.q.out @@ -78,8 +78,8 @@ POSTHOOK: query: SELECT a, b, c FROM cmv_mat_view_n2 POSTHOOK: type: QUERY POSTHOOK: Input: default@cmv_mat_view_n2 POSTHOOK: Output: hdfs://### HDFS PATH ### -2 bob 3.140000104904175 -2 bonnie 172342.203125 +2 bob 3.14 +2 bonnie 172342.2 PREHOOK: query: SHOW TBLPROPERTIES cmv_mat_view_n2 PREHOOK: type: SHOW_TBLPROPERTIES POSTHOOK: query: SHOW TBLPROPERTIES cmv_mat_view_n2 @@ -125,9 +125,9 @@ POSTHOOK: query: SELECT a, c FROM cmv_mat_view2_n0 POSTHOOK: type: QUERY POSTHOOK: Input: default@cmv_mat_view2_n0 POSTHOOK: Output: hdfs://### HDFS PATH ### -3 15.800000190734863 -3 9.800000190734863 -3 978.760009765625 +3 15.8 +3 9.8 +3 978.76 PREHOOK: query: SHOW TBLPROPERTIES cmv_mat_view2_n0 PREHOOK: type: SHOW_TBLPROPERTIES POSTHOOK: query: SHOW TBLPROPERTIES cmv_mat_view2_n0 @@ -193,9 +193,9 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@cmv_basetable_n2 POSTHOOK: Input: default@cmv_mat_view2_n0 POSTHOOK: Output: hdfs://### HDFS PATH ### -3 15.800000190734863 -3 9.800000190734863 -3 978.760009765625 +3 15.8 +3 9.8 +3 978.76 Warning: Shuffle Join MERGEJOIN[10][tables = [cmv_mat_view2_n0, $hdt$_0]] in Stage 'Reducer 2' is a cross product PREHOOK: query: EXPLAIN SELECT * FROM ( @@ -312,9 +312,9 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@cmv_basetable_n2 POSTHOOK: Input: default@cmv_mat_view2_n0 POSTHOOK: Output: hdfs://### HDFS PATH ### -3 15.800000190734863 3 978.76 -3 9.800000190734863 3 978.76 -3 978.760009765625 3 978.76 +3 15.8 3 978.76 +3 9.8 3 978.76 +3 978.76 3 978.76 PREHOOK: query: INSERT INTO cmv_basetable_n2 VALUES (cast(current_timestamp() AS timestamp), 3, 'charlie', 'charlie_c', 15.8, 1) PREHOOK: type: QUERY @@ -675,10 +675,10 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@cmv_basetable_n2 POSTHOOK: Input: default@cmv_mat_view2_n0 POSTHOOK: Output: hdfs://### HDFS PATH ### -3 15.800000190734863 3 978.76 -3 15.800000190734863 3 978.76 -3 9.800000190734863 3 978.76 -3 978.760009765625 3 978.76 +3 15.8 3 978.76 +3 15.8 3 978.76 +3 9.8 3 978.76 +3 978.76 3 978.76 PREHOOK: query: DROP MATERIALIZED VIEW cmv_mat_view_n2 PREHOOK: type: DROP_MATERIALIZED_VIEW PREHOOK: Input: default@cmv_mat_view_n2 diff --git a/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out b/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out index 7aa1763161b525161bafa0834a4026ecd19bb356..395a69be238dca92fd47ed267f6de93290b22be7 100644 --- a/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out +++ b/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out @@ -75,7 +75,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -39590.24724686146 2.7308662809692383E7 -39967 7781089 1408069801800 10992545287 +1969-01-01 00:00:00.0 US/Pacific -39590.24724686146 2.7308662793799996E7 -39967 7781089 1408069801800 10992545287 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), MIN(cfloat), MIN(cdouble), MIN(ctinyint), MIN(csmallint),MIN(cint), MIN(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -116,7 +116,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -1790.7781 -308691.84375 2 14255 -1073279343 -8577981133 +1969-01-01 00:00:00.0 US/Pacific -1790.7781 -308691.8399999999 2 14255 -1073279343 -8577981133 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), MAX(cfloat), MAX(cdouble), MAX(ctinyint), MAX(csmallint),MAX(cint), MAX(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -157,7 +157,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific 769.16394 1.9565518E7 -45 -8101 1276572707 4923772860 +1969-01-01 00:00:00.0 US/Pacific 769.16394 1.9565517977799997E7 -45 -8101 1276572707 4923772860 PREHOOK: query: EXPLAIN SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -194,8 +194,8 @@ POSTHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1cGVWH7n1QU -596096.6875 -821UdmGbkEf4j -14161.827026367188 +1cGVWH7n1QU -596096.6799999999 +821UdmGbkEf4j -14161.82699999999 00iT08 0.0 02v8WnLuYDos3Cq 0.0 yv1js 0.0 @@ -240,7 +240,7 @@ POSTHOOK: query: SELECT cstring2, MAX(cdouble) FROM druid_table_alltypesorc GROU POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -NULL 1.9565518E7 +NULL 1.9565517977799997E7 0034fkcXMQI3 15601.0 004J8y 0.0 00GNm -200.0 diff --git a/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out b/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out index 9c412d97dd4d42e7e45990fa3be380f947103cfd..917e22a32a26a9ccc2b09a1d685d44f51b6693b7 100644 --- a/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out +++ b/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out @@ -17,7 +17,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -39590.24724686146 2.7308662809692383E7 -39967 7781089 1408069801800 10992545287 +1969-01-01 00:00:00.0 US/Pacific -39590.24724686146 2.7308662793799996E7 -39967 7781089 1408069801800 10992545287 PREHOOK: query: SELECT floor_year(`__time`), MIN(cfloat), MIN(cdouble), MIN(ctinyint), MIN(csmallint),MIN(cint), MIN(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -28,7 +28,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -1790.7781 -308691.84375 2 14255 -1073279343 -8577981133 +1969-01-01 00:00:00.0 US/Pacific -1790.7781 -308691.8399999999 2 14255 -1073279343 -8577981133 PREHOOK: query: SELECT floor_year(`__time`), MAX(cfloat), MAX(cdouble), MAX(ctinyint), MAX(csmallint),MAX(cint), MAX(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -39,7 +39,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific 769.16394 1.9565518E7 -45 -8101 1276572707 4923772860 +1969-01-01 00:00:00.0 US/Pacific 769.16394 1.9565517977799997E7 -45 -8101 1276572707 4923772860 PREHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -48,8 +48,8 @@ POSTHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1cGVWH7n1QU -596096.6875 -821UdmGbkEf4j -14161.827026367188 +1cGVWH7n1QU -596096.6799999999 +821UdmGbkEf4j -14161.82699999999 00iT08 0.0 02v8WnLuYDos3Cq 0.0 yv1js 0.0 @@ -66,7 +66,7 @@ POSTHOOK: query: SELECT cstring2, MAX(cdouble) FROM druid_table_alltypesorc GROU POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -NULL 1.9565518E7 +NULL 1.9565517977799997E7 0034fkcXMQI3 15601.0 004J8y 0.0 00GNm -200.0 -- 2.15.2 (Apple Git-101.1) From 4c92e2c9ed12823420e5e46e06bbf7fc6826e013 Mon Sep 17 00:00:00 2001 From: Nishant Date: Wed, 3 Oct 2018 20:01:56 +0530 Subject: [PATCH 2/2] [HIVE-20683] Add the Ability to push Dynamic Between and Bloom filters to Druid --- .../datasets/druid_table_alltypesorc/load.hive.sql | 10 +- druid-handler/pom.xml | 6 +- .../hadoop/hive/druid/DruidStorageHandler.java | 38 + .../hive/druid/DruidStorageHandlerUtils.java | 312 +++- .../hive/druid/security/ResponseCookieHandler.java | 8 +- .../RetryIfUnauthorizedResponseHandler.java | 8 +- .../druid/serde/DruidGroupByQueryRecordReader.java | 6 +- .../hive/druid/serde/DruidQueryRecordReader.java | 185 +- .../druid/serde/DruidScanQueryRecordReader.java | 7 +- .../druid/serde/DruidSelectQueryRecordReader.java | 6 +- .../serde/DruidTimeseriesQueryRecordReader.java | 6 +- .../druid/serde/DruidTopNQueryRecordReader.java | 6 +- itests/qtest-druid/pom.xml | 5 + .../hive/ql/metadata/HiveStorageHandler.java | 14 + .../apache/hadoop/hive/ql/plan/TableScanDesc.java | 28 +- .../druidmini_semijoin_reduction_all_types.q | 144 ++ .../druidmini_semijoin_reduction_all_types.q.out | 1937 ++++++++++++++++++++ 17 files changed, 2619 insertions(+), 107 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q create mode 100644 ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out diff --git a/data/files/datasets/druid_table_alltypesorc/load.hive.sql b/data/files/datasets/druid_table_alltypesorc/load.hive.sql index 5fde266a014766eb0ddca85006b80cecea9dd540..dc730e0e10d5a1d7e77eae4888e7daee8b91313c 100644 --- a/data/files/datasets/druid_table_alltypesorc/load.hive.sql +++ b/data/files/datasets/druid_table_alltypesorc/load.hive.sql @@ -10,7 +10,10 @@ CREATE TABLE alltypesorc1( ctimestamp1 TIMESTAMP, ctimestamp2 TIMESTAMP, cboolean1 BOOLEAN, - cboolean2 BOOLEAN) + cboolean2 BOOLEAN, + cintstring STRING, + cfloatstring STRING, + cdoublestring STRING) STORED AS ORC; LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/alltypesorc" @@ -30,7 +33,10 @@ SELECT cast (`ctimestamp1` as timestamp with local time zone) as `__time`, cint, cbigint, cboolean1, - cboolean2 + cboolean2, + cast(cint as string) as cintstring, + cast(cfloat as string) as cfloatstring, + cast(cdouble as string) as cdoublestring FROM alltypesorc1 where ctimestamp1 IS NOT NULL; DROP TABLE alltypesorc1; diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml index c00884ea1ce8f81131c12e51098daad2fa7615d8..12cda595c2cb77ba426310f4ecc23ecee9b5fec7 100644 --- a/druid-handler/pom.xml +++ b/druid-handler/pom.xml @@ -170,6 +170,11 @@ postgresql-metadata-storage ${druid.version} + + org.apache.druid.extensions + druid-bloom-filter + ${druid.version} + org.apache.hadoop hadoop-common @@ -323,7 +328,6 @@ org.apache.druid:* org.apache.druid.extensions:* - com.metamx:* io.netty:* org.apache.calcite:* com.fasterxml.jackson.core:* diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java index d39d33fef5bd5462c8a2e6059f5a60c9310e44af..7d62d530ee264c00374306c8d92c9a0b5e1caffc 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.lang3.StringUtils; import org.apache.druid.data.input.impl.DimensionSchema; @@ -54,6 +55,8 @@ import org.apache.druid.metadata.storage.mysql.MySQLConnectorConfig; import org.apache.druid.metadata.storage.postgresql.PostgreSQLConnector; import org.apache.druid.metadata.storage.postgresql.PostgreSQLConnectorConfig; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.Query; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -89,7 +92,10 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider; import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; import org.apache.hadoop.hive.ql.session.SessionState; @@ -1152,4 +1158,36 @@ public StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaExceptio // e.g. Total size of segments in druid, loadstatus of table on historical nodes etc. return null; } + + @Override + public Map getOperatorDescProperties(OperatorDesc operatorDesc, + Map initialProps + ) { + if(operatorDesc instanceof TableScanDesc) { + TableScanDesc tableScanDesc = (TableScanDesc) operatorDesc; + ExprNodeGenericFuncDesc filterExpr = tableScanDesc.getFilterExpr(); + String druidQuery = initialProps.get(Constants.DRUID_QUERY_JSON); + + if(filterExpr != null && druidQuery != null) { + try { + Query query = DruidStorageHandlerUtils.JSON_MAPPER.readValue( + druidQuery, BaseQuery.class); + Query queryWithDynamicFilters = DruidStorageHandlerUtils + .addDynamicFilters(query, filterExpr, conf, false); + Map props = Maps.newHashMap(initialProps); + props.put(Constants.DRUID_QUERY_JSON, + DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(queryWithDynamicFilters) + ); + return props; + } catch (IOException e) { + LOG.error( + "Exception while deserializing druid query. Explain plan may not have final druid query", + e + ); + } + } + } + // Case when we do not have any additional info to add. + return initialProps; + } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java index 26321e1560d5a284766eb527ea5eab7db0fbbbf1..63dc4088da7ddb6efa49cf45cad649e7be85412f 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java @@ -25,15 +25,23 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Interner; import com.google.common.collect.Interners; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.io.CharStreams; - +import org.apache.calcite.adapter.druid.DruidQuery; +import org.apache.calcite.sql.validate.SqlValidatorUtil; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.guice.BloomFilterSerializersModule; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.MapUtils; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -42,15 +50,11 @@ import org.apache.druid.java.util.http.client.response.FullResponseHandler; import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; -import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.SQLMetadataConnector; import org.apache.druid.metadata.storage.mysql.MySQLConnector; +import org.apache.druid.query.Druids; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; @@ -64,18 +68,34 @@ import org.apache.druid.query.expression.TimestampParseExprMacro; import org.apache.druid.query.expression.TimestampShiftExprMacro; import org.apache.druid.query.expression.TrimExprMacro; +import org.apache.druid.query.filter.AndDimFilter; +import org.apache.druid.query.filter.BloomDimFilter; +import org.apache.druid.query.filter.BoundDimFilter; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.ordering.StringComparator; +import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.select.SelectQuery; import org.apache.druid.query.select.SelectQueryConfig; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.storage.hdfs.HdfsDataSegmentPusher; import org.apache.druid.storage.hdfs.HdfsDataSegmentPusherConfig; @@ -91,16 +111,37 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.io.NonSyncByteArrayInputStream; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.ExprNodeDynamicValueEvaluator; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.exec.UDF; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.UDFToDouble; +import org.apache.hadoop.hive.ql.udf.UDFToFloat; +import org.apache.hadoop.hive.ql.udf.UDFToLong; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInBloomFilter; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToString; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryProxy; import org.apache.hadoop.util.StringUtils; +import org.apache.hive.common.util.BloomKFilter; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -133,6 +174,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; @@ -140,7 +182,7 @@ import java.util.TimeZone; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; - +import java.util.stream.Collectors; /** * Utils class for Druid storage handler. @@ -202,6 +244,12 @@ // Register the shard sub type to be used by the mapper JSON_MAPPER.registerSubtypes(new NamedType(LinearShardSpec.class, "linear")); JSON_MAPPER.registerSubtypes(new NamedType(NumberedShardSpec.class, "numbered")); + + // Register Bloom Filter Serializers + BloomFilterSerializersModule bloomFilterSerializersModule = new BloomFilterSerializersModule(); + JSON_MAPPER.registerModule(bloomFilterSerializersModule); + SMILE_MAPPER.registerModule(bloomFilterSerializersModule); + // set the timezone of the object mapper // THIS IS NOT WORKING workaround is to set it as part of java opts -Duser.timezone="UTC" JSON_MAPPER.setTimeZone(TimeZone.getTimeZone("UTC")); @@ -912,4 +960,254 @@ public static IndexSpec getIndexSpec(Configuration jc) { return Pair.of(dimensions, aggregatorFactories.toArray(new AggregatorFactory[aggregatorFactories.size()])); } + + // Druid only supports String,Long,Float,Double selectors + private static Set DRUID_SUPPORTED_TYPE_INFOS = ImmutableSet.of( + TypeInfoFactory.stringTypeInfo, TypeInfoFactory.charTypeInfo, + TypeInfoFactory.varcharTypeInfo, TypeInfoFactory.byteTypeInfo, + TypeInfoFactory.intTypeInfo, TypeInfoFactory.longTypeInfo, + TypeInfoFactory.shortTypeInfo, TypeInfoFactory.doubleTypeInfo + ); + + private static Set STRING_TYPE_INFOS = ImmutableSet.of( + TypeInfoFactory.stringTypeInfo, + TypeInfoFactory.charTypeInfo, TypeInfoFactory.varcharTypeInfo + ); + + + public static org.apache.druid.query.Query addDynamicFilters(org.apache.druid.query.Query query, + ExprNodeGenericFuncDesc filterExpr, Configuration conf, boolean resolveDynamicValues + ) { + List virtualColumns = Arrays + .asList(getVirtualColumns(query).getVirtualColumns()); + org.apache.druid.query.Query rv = query; + DimFilter joinReductionFilter = toDruidFilter(filterExpr, conf, virtualColumns, + resolveDynamicValues + ); + if(joinReductionFilter != null) { + String type = query.getType(); + DimFilter filter = new AndDimFilter(joinReductionFilter, query.getFilter()); + switch (type) { + case org.apache.druid.query.Query.TIMESERIES: + rv = Druids.TimeseriesQueryBuilder.copy((TimeseriesQuery) query) + .filters(filter) + .virtualColumns(VirtualColumns.create(virtualColumns)) + .build(); + break; + case org.apache.druid.query.Query.TOPN: + rv = new TopNQueryBuilder((TopNQuery) query) + .filters(filter) + .virtualColumns(VirtualColumns.create(virtualColumns)) + .build(); + break; + case org.apache.druid.query.Query.GROUP_BY: + rv = new GroupByQuery.Builder((GroupByQuery) query) + .setDimFilter(filter) + .setVirtualColumns(VirtualColumns.create(virtualColumns)) + .build(); + break; + case org.apache.druid.query.Query.SCAN: + rv = ScanQuery.ScanQueryBuilder.copy((ScanQuery) query) + .filters(filter) + .virtualColumns(VirtualColumns.create(virtualColumns)) + .build(); + break; + case org.apache.druid.query.Query.SELECT: + rv = Druids.SelectQueryBuilder.copy((SelectQuery) query) + .filters(filter) + .virtualColumns(VirtualColumns.create(virtualColumns)) + .build(); + } + } + return rv; + } + + private static DimFilter toDruidFilter(ExprNodeDesc filterExpr, Configuration configuration, + List virtualColumns, boolean resolveDynamicValues + ) { + if(filterExpr == null) { + return null; + } + Class genericUDFClass = getGenericUDFClassFromExprDesc(filterExpr); + if(FunctionRegistry.isOpAnd(filterExpr)) { + Iterator iterator = filterExpr.getChildren().iterator(); + List delegates = Lists.newArrayList(); + while (iterator.hasNext()) { + DimFilter filter = toDruidFilter(iterator.next(), configuration, virtualColumns, + resolveDynamicValues + ); + if(filter != null) { + delegates.add(filter); + } + } + if(delegates != null && !delegates.isEmpty()) { + return new AndDimFilter(delegates); + } + } + if(FunctionRegistry.isOpOr(filterExpr)) { + Iterator iterator = filterExpr.getChildren().iterator(); + List delegates = Lists.newArrayList(); + while (iterator.hasNext()) { + DimFilter filter = toDruidFilter(iterator.next(), configuration, virtualColumns, + resolveDynamicValues + ); + if(filter != null) { + delegates.add(filter); + } + } + if(delegates != null) { + return new OrDimFilter(delegates); + } + } else if(GenericUDFBetween.class == genericUDFClass) { + List child = filterExpr.getChildren(); + String col = extractColName(child.get(1), virtualColumns); + if(col != null) { + try { + StringComparator comparator = STRING_TYPE_INFOS.contains(child.get(1).getTypeInfo()) + ? StringComparators.LEXICOGRAPHIC + : StringComparators.NUMERIC; + String lower = evaluate(child.get(2), configuration, resolveDynamicValues); + String upper = evaluate(child.get(3), configuration, resolveDynamicValues); + return new BoundDimFilter(col, lower, upper, false, false, null, null, + StringComparator.fromString("numeric") + ); + + } catch (HiveException e) { + throw new RuntimeException(e); + } + } + } else if(GenericUDFInBloomFilter.class == genericUDFClass) { + List child = filterExpr.getChildren(); + String col = extractColName(child.get(0), virtualColumns); + if(col != null) { + try { + BloomKFilter bloomFilter = evaluateBloomFilter(child.get(1), configuration, + resolveDynamicValues + ); + return new BloomDimFilter(col, bloomFilter, null); + } catch (HiveException e) { + throw new RuntimeException(e); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + return null; + } + + private static String evaluate(ExprNodeDesc desc, Configuration configuration, + boolean resolveDynamicValue + ) throws HiveException { + ExprNodeEvaluator exprNodeEvaluator = ExprNodeEvaluatorFactory.get(desc, configuration); + if(exprNodeEvaluator instanceof ExprNodeDynamicValueEvaluator && !resolveDynamicValue) { + return desc.getExprStringForExplain(); + } else { + return exprNodeEvaluator.evaluate(null).toString(); + } + } + + private static BloomKFilter evaluateBloomFilter(ExprNodeDesc desc, Configuration configuration, + boolean resolveDynamicValue + ) + throws HiveException, IOException { + if(!resolveDynamicValue) { + // return a dummy bloom filter for explain + return new BloomKFilter(1); + } else { + BytesWritable bw = (BytesWritable) ExprNodeEvaluatorFactory.get(desc, configuration) + .evaluate(null); + byte[] bytes = new byte[bw.getLength()]; + System.arraycopy(bw.getBytes(), 0, bytes, 0, bw.getLength()); + InputStream in = new NonSyncByteArrayInputStream(bytes); + return BloomKFilter.deserialize(in); + } + } + + public static String extractColName(ExprNodeDesc expr, List virtualColumns) { + if(!DRUID_SUPPORTED_TYPE_INFOS.contains(expr.getTypeInfo())) { + // This column type is currently not supported in druid.(e.g boolean) + // We cannot pass the bloom filter to druid since bloom filter tests for exact object bytes. + return null; + } + if(expr instanceof ExprNodeColumnDesc) { + return ((ExprNodeColumnDesc) expr).getColumn(); + } + + ExprNodeGenericFuncDesc funcDesc = null; + if(expr instanceof ExprNodeGenericFuncDesc) { + funcDesc = (ExprNodeGenericFuncDesc) expr; + } + if(null == funcDesc) { + return null; + } + GenericUDF udf = funcDesc.getGenericUDF(); + // bail out if its not a simple cast expression. + if(funcDesc.getChildren().size() == 1 && funcDesc.getChildren() + .get(0) instanceof ExprNodeColumnDesc) { + return null; + } + String columnName = ((ExprNodeColumnDesc) (funcDesc.getChildren() + .get(0))).getColumn(); + ValueType targetType = null; + if(udf instanceof GenericUDFBridge) { + Class udfClass = ((GenericUDFBridge) udf).getUdfClass(); + if(udfClass.equals(UDFToDouble.class)) { + targetType = ValueType.DOUBLE; + } else if(udfClass.equals(UDFToFloat.class)) { + targetType = ValueType.FLOAT; + } else if(udfClass.equals(UDFToLong.class)) { + targetType = ValueType.LONG; + } else if(udfClass.equals(GenericUDFToString.class)) { + targetType = ValueType.STRING; + } + } + + if(targetType == null) { + return null; + } + String virtualColumnExpr = DruidQuery + .format("CAST(%s, '%s')", columnName, targetType.toString()); + for(VirtualColumn column : virtualColumns) { + if(column instanceof ExpressionVirtualColumn && ((ExpressionVirtualColumn) column) + .getExpression().equals(virtualColumnExpr)) { + // Found an existing virtual column with same expression, no need to add another virtual column + return column.getOutputName(); + } + } + Set usedColumnNames = virtualColumns.stream().map(col -> col.getOutputName()) + .collect(Collectors.toSet()); + final String name = SqlValidatorUtil + .uniquify("vc", usedColumnNames, SqlValidatorUtil.EXPR_SUGGESTER); + ExpressionVirtualColumn expressionVirtualColumn = new ExpressionVirtualColumn(name, + virtualColumnExpr, targetType, null + ); + virtualColumns.add(expressionVirtualColumn); + return name; + } + + public static VirtualColumns getVirtualColumns(org.apache.druid.query.Query query) { + String type = query.getType(); + switch (type) { + case org.apache.druid.query.Query.TIMESERIES: + return ((TimeseriesQuery) query).getVirtualColumns(); + case org.apache.druid.query.Query.TOPN: + return ((TopNQuery) query).getVirtualColumns(); + case org.apache.druid.query.Query.GROUP_BY: + return ((GroupByQuery) query).getVirtualColumns(); + case org.apache.druid.query.Query.SCAN: + return ((ScanQuery) query).getVirtualColumns(); + case org.apache.druid.query.Query.SELECT: + return ((SelectQuery) query).getVirtualColumns(); + } + throw new UnsupportedOperationException("Unsupported Query type" + query); + } + + private static Class getGenericUDFClassFromExprDesc(ExprNodeDesc desc) { + if(!(desc instanceof ExprNodeGenericFuncDesc)) { + return null; + } + ExprNodeGenericFuncDesc genericFuncDesc = (ExprNodeGenericFuncDesc) desc; + return genericFuncDesc.getGenericUDF().getClass(); + } + } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java index 2b963a59c0cb9238db8e811c61093683edd8088b..24facfc1f8210e9df16e0d3e6e100f9324418744 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java @@ -47,7 +47,7 @@ public ResponseCookieHandler(URI uri, CookieManager manager, HttpResponseHandler } @Override - public ClientResponse handleResponse(HttpResponse httpResponse) + public ClientResponse handleResponse(HttpResponse httpResponse, TrafficCop trafficCop) { try { final HttpHeaders headers = httpResponse.headers(); @@ -57,16 +57,16 @@ public ResponseCookieHandler(URI uri, CookieManager manager, HttpResponseHandler log.error("Error while processing Cookies from header", e); } finally { - return delegate.handleResponse(httpResponse); + return delegate.handleResponse(httpResponse, trafficCop); } } @Override public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk httpChunk + ClientResponse clientResponse, HttpChunk httpChunk, long chunkNum ) { - return delegate.handleChunk(clientResponse, httpChunk); + return delegate.handleChunk(clientResponse, httpChunk, chunkNum); } @Override diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java index 4057739cd30219b40dfa6d46386ccdc359486537..faee9b4b39885b2440c79943de31087bb68fb882 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java @@ -41,7 +41,7 @@ public RetryIfUnauthorizedResponseHandler( } @Override - public ClientResponse> handleResponse(HttpResponse httpResponse) + public ClientResponse> handleResponse(HttpResponse httpResponse, TrafficCop trafficCop) { log.debug("UnauthorizedResponseHandler - Got response status {}", httpResponse.getStatus()); if (httpResponse.getStatus().equals(HttpResponseStatus.UNAUTHORIZED)) { @@ -49,20 +49,20 @@ public RetryIfUnauthorizedResponseHandler( httpResponse.getContent().toString(); return ClientResponse.unfinished(RetryResponseHolder.retry()); } else { - return wrap(httpResponseHandler.handleResponse(httpResponse)); + return wrap(httpResponseHandler.handleResponse(httpResponse, trafficCop)); } } @Override public ClientResponse> handleChunk( - ClientResponse> clientResponse, HttpChunk httpChunk + ClientResponse> clientResponse, HttpChunk httpChunk, long chunkNum ) { if (clientResponse.getObj().shouldRetry()) { httpChunk.getContent().toString(); return clientResponse; } else { - return wrap(httpResponseHandler.handleChunk(unwrap(clientResponse), httpChunk)); + return wrap(httpResponseHandler.handleChunk(unwrap(clientResponse), httpChunk, chunkNum)); } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java index c49a6e3358bca6d89b586f84407ece5eb1b0f076..26d4324e3aa55185440c710f79367db16b1a6a5c 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java @@ -59,8 +59,8 @@ @Override public boolean nextKeyValue() { // Results - if (queryResultsIterator.hasNext()) { - final Row row = queryResultsIterator.next(); + if (getQueryResultsIterator().hasNext()) { + final Row row = getQueryResultsIterator().next(); // currently druid supports only MapBasedRow as Jackson SerDe so it should safe to cast without check currentRow = (MapBasedRow) row; currentEvent = currentRow.getEvent(); @@ -101,7 +101,7 @@ } @Override public float getProgress() throws IOException { - return queryResultsIterator.hasNext() ? 0 : 1; + return getQueryResultsIterator().hasNext() ? 0 : 1; } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java index 65585d016c2160b84866f81d6ccb2ef36a4c2fda..16d3261a6ec46a1affb8b2353f430c83a0a5a667 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java @@ -24,6 +24,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import org.apache.calcite.adapter.druid.DruidQuery; +import org.apache.calcite.sql.validate.SqlValidatorUtil; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.guava.CloseQuietly; @@ -31,16 +34,58 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.filter.AndDimFilter; +import org.apache.druid.query.filter.BloomDimFilter; +import org.apache.druid.query.filter.BoundDimFilter; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.ordering.StringComparator; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.select.SelectQuery; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.io.NonSyncByteArrayInputStream; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.druid.DruidStorageHandler; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.hive.druid.io.HiveDruidSplit; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.ql.udf.UDFToDouble; +import org.apache.hadoop.hive.ql.udf.UDFToFloat; +import org.apache.hadoop.hive.ql.udf.UDFToLong; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInBloomFilter; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToString; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hive.common.util.BloomKFilter; import org.apache.parquet.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,9 +93,13 @@ import java.io.Closeable; import java.io.IOException; import java.io.InputStream; +import java.util.Arrays; import java.util.Iterator; +import java.util.List; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.stream.Collectors; /** * Base record reader for given a Druid query. This class contains the logic to @@ -71,6 +120,8 @@ private ObjectMapper mapper; // Smile mapper is used to read query results that are serialized as binary instead of json private ObjectMapper smileMapper; + private Configuration conf; + private String address; /** * Query that Druid executes. @@ -80,7 +131,37 @@ /** * Query results as a streaming iterator. */ - protected JsonParserIterator queryResultsIterator = null; + private JsonParserIterator queryResultsIterator = null; + + public JsonParserIterator getQueryResultsIterator() + { + if(this.queryResultsIterator != null){ + return queryResultsIterator; + } else { + String filterExprSerialized = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR); + if (filterExprSerialized != null) { + ExprNodeGenericFuncDesc filterExpr = SerializationUtilities + .deserializeExpression(filterExprSerialized); + query = DruidStorageHandlerUtils.addDynamicFilters(query, filterExpr, conf, true); + } + Request request = null; + try { + request = DruidStorageHandlerUtils.createSmileRequest(address, query); + LOG.info("Retrieving data from druid using query:\n " + query); + Future inputStreamFuture = this.httpClient + .go(request, new InputStreamResponseHandler()); + queryResultsIterator = new JsonParserIterator(this.smileMapper, resultsType, inputStreamFuture, + request.getUrl().toString(), query + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return queryResultsIterator; + } + } + + /** * Result type definition used to read the rows, this is query dependent. @@ -95,6 +176,7 @@ public void initialize(InputSplit split, TaskAttemptContext context) throws IOEx public void initialize(InputSplit split, Configuration conf, ObjectMapper mapper, ObjectMapper smileMapper, HttpClient httpClient ) throws IOException { + this.conf = conf; HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split; Preconditions.checkNotNull(hiveDruidSplit, "input split is null ???"); this.mapper = Preconditions.checkNotNull(mapper, "object Mapper can not be null"); @@ -102,55 +184,14 @@ public void initialize(InputSplit split, Configuration conf, ObjectMapper mapper this.smileMapper = Preconditions.checkNotNull(smileMapper, "Smile Mapper can not be null"); // Create query this.query = this.mapper.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class); + Preconditions.checkNotNull(query); this.resultsType = getResultTypeDef(); this.httpClient = Preconditions.checkNotNull(httpClient, "need Http Client"); - final String[] locations = hiveDruidSplit.getLocations(); - boolean initlialized = false; - int currentLocationIndex = 0; - Exception ex = null; - while (!initlialized && currentLocationIndex < locations.length) { - String address = locations[currentLocationIndex++]; - if(Strings.isNullOrEmpty(address)) { - throw new IOException("can not fetch results from empty or null host value"); - } - // Execute query - LOG.debug("Retrieving data from druid location[{}] using query:[{}] ", address, query); - try { - Request request = DruidStorageHandlerUtils.createSmileRequest(address, query); - Future inputStreamFuture = this.httpClient - .go(request, new InputStreamResponseHandler()); - queryResultsIterator = new JsonParserIterator(this.smileMapper, resultsType, - inputStreamFuture, request.getUrl().toString(), query - ); - queryResultsIterator.init(); - initlialized = true; - } catch (IOException | ExecutionException | InterruptedException e) { - if(queryResultsIterator != null) { - // We got exception while querying results from this host. - queryResultsIterator.close(); - } - LOG.error("Failure getting results for query[{}] from host[{}] because of [{}]", - query, - address, - e.getMessage() - ); - if(ex == null) { - ex = e; - } else { - ex.addSuppressed(e); - } - } - } - if(!initlialized) { - throw new RE( - ex, - "Failure getting results for query[%s] from locations[%s] because of [%s]", - query, - locations, - ex.getMessage() - ); + address = hiveDruidSplit.getLocations()[0]; + if (Strings.isNullOrEmpty(address)) { + throw new IOException("can not fetch results form empty or null host value"); } } @@ -198,7 +239,9 @@ public long getPos() { @Override public void close() { - CloseQuietly.close(queryResultsIterator); + if(queryResultsIterator != null) { + CloseQuietly.close(queryResultsIterator); + } } /** @@ -242,6 +285,8 @@ public JsonParserIterator(ObjectMapper mapper, @Override public boolean hasNext() { + init(); + if (jp.isClosed()) { return false; } @@ -256,6 +301,8 @@ public boolean hasNext() @Override public R next() { + init(); + try { final R retVal = objectCodec.readValue(jp, typeRef); jp.nextToken(); @@ -272,23 +319,35 @@ public void remove() throw new UnsupportedOperationException(); } - private void init() throws IOException, ExecutionException, InterruptedException { - if(jp == null) { - InputStream is = future.get(); - if(is == null) { - throw new IOException(String.format("query[%s] url[%s] timed out", query, url)); - } else { - jp = mapper.getFactory().createParser(is).configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, true); + private void init() + { + if (jp == null) { + try { + InputStream is = future.get(); + if (is == null) { + throw new IOException(String.format("query[%s] url[%s] timed out", query, url)); + } else { + jp = mapper.getFactory().createParser(is).configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, true); + } + final JsonToken nextToken = jp.nextToken(); + if (nextToken == JsonToken.START_OBJECT) { + QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class); + throw new QueryInterruptedException(cause); + } else if (nextToken != JsonToken.START_ARRAY) { + throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url); + } else { + jp.nextToken(); + objectCodec = jp.getCodec(); + } } - final JsonToken nextToken = jp.nextToken(); - if(nextToken == JsonToken.START_OBJECT) { - QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class); - throw new QueryInterruptedException(cause); - } else if(nextToken != JsonToken.START_ARRAY) { - throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url); - } else { - jp.nextToken(); - objectCodec = jp.getCodec(); + catch (IOException | InterruptedException | ExecutionException e) { + throw new RE( + e, + "Failure getting results for query[%s] url[%s] because of [%s]", + query, + url, + e.getMessage() + ); } } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java index 9cd40fa0819a5b6dc4ef96e6616c37fdd34d3b17..c53f5f776c54b34a91ecee8d8ab210fbd104118d 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java @@ -55,8 +55,8 @@ public boolean nextKeyValue() throws IOException { if (compactedValues.hasNext()) { return true; } - if (queryResultsIterator.hasNext()) { - current = queryResultsIterator.next(); + if (getQueryResultsIterator().hasNext()) { + current = getQueryResultsIterator().next(); compactedValues = ((List>) current.getEvents()).iterator(); return nextKeyValue(); } @@ -94,7 +94,6 @@ public boolean next(NullWritable key, DruidWritable value) throws IOException { @Override public float getProgress() { - return queryResultsIterator.hasNext() || compactedValues.hasNext() ? 0 : 1; + return getQueryResultsIterator().hasNext() || compactedValues.hasNext() ? 0 : 1; } - } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java index 3d9c5a8b4393ac5013d056321a0d2688f4f93953..7def5899183552692aa05c32a252353c3ea91f1c 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java @@ -57,8 +57,8 @@ public boolean nextKeyValue() throws IOException { if (values.hasNext()) { return true; } - if (queryResultsIterator.hasNext()) { - current = queryResultsIterator.next(); + if (getQueryResultsIterator().hasNext()) { + current = getQueryResultsIterator().next(); values = current.getValue().getEvents().iterator(); return nextKeyValue(); } @@ -95,7 +95,7 @@ public boolean next(NullWritable key, DruidWritable value) throws IOException { @Override public float getProgress() { - return queryResultsIterator.hasNext() || values.hasNext() ? 0 : 1; + return getQueryResultsIterator().hasNext() || values.hasNext() ? 0 : 1; } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java index 3f82428ffe96f7e66eef183e42ff44d117eff52a..5be2f5acf2fe457442db236b4ab8323e209763d0 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java @@ -44,8 +44,8 @@ protected JavaType getResultTypeDef() { @Override public boolean nextKeyValue() { - if (queryResultsIterator.hasNext()) { - current = queryResultsIterator.next(); + if (getQueryResultsIterator().hasNext()) { + current = getQueryResultsIterator().next(); return true; } return false; @@ -83,7 +83,7 @@ public boolean next(NullWritable key, DruidWritable value) { @Override public float getProgress() throws IOException { - return queryResultsIterator.hasNext() ? 0 : 1; + return getQueryResultsIterator().hasNext() ? 0 : 1; } } diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java index 58398b3ef6247c1eaba1aa27892e0fced86fe10b..396abc263e070ed0c700f108e4412f9e81d99fd4 100644 --- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java +++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java @@ -59,8 +59,8 @@ public boolean nextKeyValue() { if (values.hasNext()) { return true; } - if (queryResultsIterator.hasNext()) { - current = queryResultsIterator.next(); + if (getQueryResultsIterator().hasNext()) { + current = getQueryResultsIterator().next(); values = current.getValue().getValue().iterator(); return nextKeyValue(); } @@ -104,7 +104,7 @@ public boolean next(NullWritable key, DruidWritable value) { @Override public float getProgress() { - return queryResultsIterator.hasNext() || values.hasNext() ? 0 : 1; + return getQueryResultsIterator().hasNext() || values.hasNext() ? 0 : 1; } } diff --git a/itests/qtest-druid/pom.xml b/itests/qtest-druid/pom.xml index 22fa21611fac921dd06b0de5b57c93e921a34218..7ba75c0c061ade8a8fefc45345bf17ffd0ce7eeb 100644 --- a/itests/qtest-druid/pom.xml +++ b/itests/qtest-druid/pom.xml @@ -104,6 +104,11 @@ + + org.apache.druid.extensions + druid-bloom-filter + ${druid.version} + org.apache.druid.extensions druid-kafka-indexing-service diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 2ebb1493545182cff8a3ed0a3a809eb835014879..7b928528ae6ba8c2e0674159c985ebb49330bbbd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; import org.apache.hadoop.hive.serde2.AbstractSerDe; @@ -172,4 +173,17 @@ public default StorageHandlerInfo getStorageHandlerInfo(Table table) throws Meta default LockType getLockType(WriteEntity writeEntity){ return LockType.EXCLUSIVE; } + + /** + * Used to add additional operator specific information from storage handler during DESCRIBE EXTENDED statement + * + * @param operatorDesc operatorDesc + * @param initialProps Map containing initial operator properties + * @return Map containing additional operator specific information from storage handler + * OR `initialProps` if the storage handler choose to not provide any such information. + */ + public default Map getOperatorDescProperties(OperatorDesc operatorDesc, Map initialProps) + { + return initialProps; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java index 192c04c6615c716b2b18d0d6ed522a99c15ab2eb..d8083f4120f45798f51327d042f4c133b6083f76 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java @@ -18,16 +18,9 @@ package org.apache.hadoop.hive.ql.plan; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; - import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; import org.apache.hadoop.hive.ql.optimizer.signature.Signature; @@ -38,6 +31,14 @@ import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; + /** * Table Scan Descriptor Currently, data is only read from a base source as part * of map-reduce framework. So, nothing is stored in the descriptor. But, more @@ -471,9 +472,16 @@ public boolean isNeedSkipHeaderFooters() { } @Override - @Explain(displayName = "properties", explainLevels = { Level.DEFAULT, Level.USER, Level.EXTENDED }) public Map getOpProps() { - return opProps; + return opProps; + } + + @Explain(displayName = "properties", explainLevels = { Level.DEFAULT, Level.USER, Level.EXTENDED }) + public Map getOpPropsWithStorageHandlerProps() { + HiveStorageHandler storageHandler = tableMetadata.getStorageHandler(); + return storageHandler == null + ? opProps + : storageHandler.getOperatorDescProperties(this, opProps); } public class TableScanOperatorExplainVectorization extends OperatorExplainVectorization { diff --git a/ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q b/ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q new file mode 100644 index 0000000000000000000000000000000000000000..10b56c364e4c8c96c424b603702e528ae71ccdca --- /dev/null +++ b/ql/src/test/queries/clientpositive/druidmini_semijoin_reduction_all_types.q @@ -0,0 +1,144 @@ +--! qt:dataset:srcpart +--! qt:dataset:druid_table_alltypesorc +--! qt:dataset:alltypesorc + +set hive.compute.query.using.stats=false; +set hive.mapred.mode=nonstrict; +set hive.explain.user=false; +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.tez.dynamic.partition.pruning=true; +set hive.tez.dynamic.semijoin.reduction=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; +set hive.stats.autogather=true; +set hive.tez.bigtable.minsize.semijoin.reduction=1; +set hive.tez.min.bloom.filter.entries=1; +set hive.stats.fetch.column.stats=true; +set hive.disable.unsafe.external.table.operations=false; +set hive.tez.dynamic.semijoin.reduction.for.mapjoin=true; + +DROP TABLE IF EXISTS alltypesorc_small; +CREATE TABLE alltypesorc_small( + ctinyint TINYINT, + csmallint SMALLINT, + cint INT, + cbigint BIGINT, + cfloat FLOAT, + cdouble DOUBLE, + cstring1 STRING, + cstring2 STRING, + ctimestamp1 TIMESTAMP, + cboolean1 BOOLEAN, + cboolean2 BOOLEAN) + STORED AS ORC; +Insert into table alltypesorc_small +Select ctinyint, csmallint, cint, cbigint, cfloat, cdouble, cstring1, cstring2, cast(`__time` as timestamp), cboolean1, cboolean2 from druid_table_alltypesorc where cstring2 like '%a%' and cstring1 like '%a%'; +Select count(*) from alltypesorc_small; +Select count(*) from druid_table_alltypesorc; + +DESCRIBE druid_table_alltypesorc; +DESCRIBE alltypesorc_small; + +-- Test Joins on all column types one by one +-- String +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1); + +-- tinyint +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint); + +-- smallint +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint); + +-- int +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint); + +-- bigint +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint); + +-- float +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat); + +-- double +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble); +set hive.disable.unsafe.external.table.operations=true; + +-- timestamp +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = druid_table_alltypesorc.`__time`); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp)); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp)); + +-- boolean +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1); + + +-- Test Casts + +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring); + + +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring); + + +set hive.disable.unsafe.external.table.operations=false; +EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring); +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring); +set hive.disable.unsafe.external.table.operations=true; +select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring); + + + + + + + + + + + + + + + diff --git a/ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out b/ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out new file mode 100644 index 0000000000000000000000000000000000000000..4a58a9e34ea78e9b838dcfe654d0c438b6b2f16d --- /dev/null +++ b/ql/src/test/results/clientpositive/druid/druidmini_semijoin_reduction_all_types.q.out @@ -0,0 +1,1937 @@ +PREHOOK: query: DROP TABLE IF EXISTS alltypesorc_small +PREHOOK: type: DROPTABLE +POSTHOOK: query: DROP TABLE IF EXISTS alltypesorc_small +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE alltypesorc_small( + ctinyint TINYINT, + csmallint SMALLINT, + cint INT, + cbigint BIGINT, + cfloat FLOAT, + cdouble DOUBLE, + cstring1 STRING, + cstring2 STRING, + ctimestamp1 TIMESTAMP, + cboolean1 BOOLEAN, + cboolean2 BOOLEAN) + STORED AS ORC +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@alltypesorc_small +POSTHOOK: query: CREATE TABLE alltypesorc_small( + ctinyint TINYINT, + csmallint SMALLINT, + cint INT, + cbigint BIGINT, + cfloat FLOAT, + cdouble DOUBLE, + cstring1 STRING, + cstring2 STRING, + ctimestamp1 TIMESTAMP, + cboolean1 BOOLEAN, + cboolean2 BOOLEAN) + STORED AS ORC +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@alltypesorc_small +PREHOOK: query: Insert into table alltypesorc_small +Select ctinyint, csmallint, cint, cbigint, cfloat, cdouble, cstring1, cstring2, cast(`__time` as timestamp), cboolean1, cboolean2 from druid_table_alltypesorc where cstring2 like '%a%' and cstring1 like '%a%' +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: default@alltypesorc_small +POSTHOOK: query: Insert into table alltypesorc_small +Select ctinyint, csmallint, cint, cbigint, cfloat, cdouble, cstring1, cstring2, cast(`__time` as timestamp), cboolean1, cboolean2 from druid_table_alltypesorc where cstring2 like '%a%' and cstring1 like '%a%' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: default@alltypesorc_small +POSTHOOK: Lineage: alltypesorc_small.cbigint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cboolean1 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cboolean2 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cboolean2, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cdouble SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cdouble, type:double, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cfloat SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cfloat, type:float, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cint, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.csmallint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:csmallint, type:smallint, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cstring1 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cstring1, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.cstring2 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:cstring2, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.ctimestamp1 SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:vc, type:timestamp, comment:from deserializer), ] +POSTHOOK: Lineage: alltypesorc_small.ctinyint SIMPLE [(druid_table_alltypesorc)druid_table_alltypesorc.FieldSchema(name:ctinyint, type:tinyint, comment:from deserializer), ] +PREHOOK: query: Select count(*) from alltypesorc_small +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) from alltypesorc_small +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: Select count(*) from druid_table_alltypesorc +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) from druid_table_alltypesorc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +9173 +PREHOOK: query: DESCRIBE druid_table_alltypesorc +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: query: DESCRIBE druid_table_alltypesorc +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@druid_table_alltypesorc +__time timestamp with local time zone from deserializer +cstring1 string from deserializer +cstring2 string from deserializer +cdouble double from deserializer +cfloat float from deserializer +ctinyint tinyint from deserializer +csmallint smallint from deserializer +cint int from deserializer +cbigint bigint from deserializer +cboolean1 boolean from deserializer +cboolean2 boolean from deserializer +cintstring string from deserializer +cfloatstring string from deserializer +cdoublestring string from deserializer +PREHOOK: query: DESCRIBE alltypesorc_small +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@alltypesorc_small +POSTHOOK: query: DESCRIBE alltypesorc_small +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@alltypesorc_small +ctinyint tinyint +csmallint smallint +cint int +cbigint bigint +cfloat float +cdouble double +cstring1 string +cstring2 string +ctimestamp1 timestamp +cboolean1 boolean +cboolean2 boolean +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cstring1 is not null (type: boolean) + Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cstring1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 6969 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (cstring1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cstring1_min) AND DynamicValue(RS_4_alltypesorc_small_cstring1_max) and in_bloom_filter(cstring1, DynamicValue(RS_4_alltypesorc_small_cstring1_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cstring1 + druid.fieldTypes string + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cstring1","lower":"DynamicValue(RS_4_alltypesorc_small_cstring1_min)","upper":"DynamicValue(RS_4_alltypesorc_small_cstring1_max)","lowerStrict":false,"upperStrict":false,"extractionFn":null,"ordering":{"type":"numeric"}},{"type":"bloom","dimension":"cstring1","bloomKFilter":"BAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA","extractionFn":null}]},{"type":"not","field":{"type":"selector","dimension":"cstring1","value":null,"extractionFn":null}}]},"columns":["cstring1"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (cstring1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cstring1_min) AND DynamicValue(RS_4_alltypesorc_small_cstring1_max) and in_bloom_filter(cstring1, DynamicValue(RS_4_alltypesorc_small_cstring1_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: cstring1 (type: string) + sort order: + + Map-reduce partition columns: cstring1 (type: string) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 cstring1 (type: string) + Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cstring1 = druid_table_alltypesorc.cstring1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ctinyint is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ctinyint (type: tinyint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: tinyint) + sort order: + + Map-reduce partition columns: _col0 (type: tinyint) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: tinyint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=3) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (ctinyint BETWEEN DynamicValue(RS_4_alltypesorc_small_ctinyint_min) AND DynamicValue(RS_4_alltypesorc_small_ctinyint_max) and in_bloom_filter(ctinyint, DynamicValue(RS_4_alltypesorc_small_ctinyint_bloom_filter))) (type: boolean) + properties: + druid.fieldNames ctinyint + druid.fieldTypes tinyint + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"ctinyint","lower":"DynamicValue(RS_4_alltypesorc_small_ctinyint_min)","upper":"DynamicValue(RS_4_alltypesorc_small_ctinyint_max)","lowerStrict":false,"upperStrict":false,"extractionFn":null,"ordering":{"type":"numeric"}},{"type":"bloom","dimension":"ctinyint","bloomKFilter":"BAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA","extractionFn":null}]},{"type":"not","field":{"type":"selector","dimension":"ctinyint","value":null,"extractionFn":null}}]},"columns":["ctinyint"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ctinyint BETWEEN DynamicValue(RS_4_alltypesorc_small_ctinyint_min) AND DynamicValue(RS_4_alltypesorc_small_ctinyint_max) and in_bloom_filter(ctinyint, DynamicValue(RS_4_alltypesorc_small_ctinyint_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ctinyint (type: tinyint) + sort order: + + Map-reduce partition columns: ctinyint (type: tinyint) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: tinyint) + 1 ctinyint (type: tinyint) + Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=3) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: tinyint), _col1 (type: tinyint), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +73895 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctinyint = druid_table_alltypesorc.ctinyint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +73895 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: csmallint is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: csmallint (type: smallint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: smallint) + sort order: + + Map-reduce partition columns: _col0 (type: smallint) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: smallint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: smallint), _col1 (type: smallint), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (csmallint BETWEEN DynamicValue(RS_4_alltypesorc_small_csmallint_min) AND DynamicValue(RS_4_alltypesorc_small_csmallint_max) and in_bloom_filter(csmallint, DynamicValue(RS_4_alltypesorc_small_csmallint_bloom_filter))) (type: boolean) + properties: + druid.fieldNames csmallint + druid.fieldTypes smallint + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"csmallint","lower":"DynamicValue(RS_4_alltypesorc_small_csmallint_min)","upper":"DynamicValue(RS_4_alltypesorc_small_csmallint_max)","lowerStrict":false,"upperStrict":false,"extractionFn":null,"ordering":{"type":"numeric"}},{"type":"bloom","dimension":"csmallint","bloomKFilter":"BAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA","extractionFn":null}]},{"type":"not","field":{"type":"selector","dimension":"csmallint","value":null,"extractionFn":null}}]},"columns":["csmallint"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (csmallint BETWEEN DynamicValue(RS_4_alltypesorc_small_csmallint_min) AND DynamicValue(RS_4_alltypesorc_small_csmallint_max) and in_bloom_filter(csmallint, DynamicValue(RS_4_alltypesorc_small_csmallint_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: csmallint (type: smallint) + sort order: + + Map-reduce partition columns: csmallint (type: smallint) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: smallint) + 1 csmallint (type: smallint) + Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: smallint), _col1 (type: smallint), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +211968 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.csmallint = druid_table_alltypesorc.csmallint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +211968 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cint is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cint (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (cint BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(cint, DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cint + druid.fieldTypes int + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cint","lower":"DynamicValue(RS_4_alltypesorc_small_cint_min)","upper":"DynamicValue(RS_4_alltypesorc_small_cint_max)","lowerStrict":false,"upperStrict":false,"extractionFn":null,"ordering":{"type":"numeric"}},{"type":"bloom","dimension":"cint","bloomKFilter":"BAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA","extractionFn":null}]},{"type":"not","field":{"type":"selector","dimension":"cint","value":null,"extractionFn":null}}]},"columns":["cint"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (cint BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(cint, DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: cint (type: int) + sort order: + + Map-reduce partition columns: cint (type: int) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 cint (type: int) + Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cbigint is not null (type: boolean) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cbigint (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (cbigint BETWEEN DynamicValue(RS_4_alltypesorc_small_cbigint_min) AND DynamicValue(RS_4_alltypesorc_small_cbigint_max) and in_bloom_filter(cbigint, DynamicValue(RS_4_alltypesorc_small_cbigint_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cbigint + druid.fieldTypes bigint + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cbigint","lower":"DynamicValue(RS_4_alltypesorc_small_cbigint_min)","upper":"DynamicValue(RS_4_alltypesorc_small_cbigint_max)","lowerStrict":false,"upperStrict":false,"extractionFn":null,"ordering":{"type":"numeric"}},{"type":"bloom","dimension":"cbigint","bloomKFilter":"BAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA","extractionFn":null}]},{"type":"not","field":{"type":"selector","dimension":"cbigint","value":null,"extractionFn":null}}]},"columns":["cbigint"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (cbigint BETWEEN DynamicValue(RS_4_alltypesorc_small_cbigint_min) AND DynamicValue(RS_4_alltypesorc_small_cbigint_max) and in_bloom_filter(cbigint, DynamicValue(RS_4_alltypesorc_small_cbigint_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: cbigint (type: bigint) + sort order: + + Map-reduce partition columns: cbigint (type: bigint) + Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 cbigint (type: bigint) + Statistics: Num rows: 10090 Data size: 76700 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cbigint = druid_table_alltypesorc.cbigint) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cfloat is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cfloat (type: float) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: float) + sort order: + + Map-reduce partition columns: _col0 (type: float) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: float) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=3) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (cfloat BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(cfloat, DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cfloat + druid.fieldTypes float + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cfloat","value":null,"extractionFn":null}},"columns":["cfloat"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (cfloat BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(cfloat, DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: cfloat (type: float) + sort order: + + Map-reduce partition columns: cfloat (type: float) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: float) + 1 cfloat (type: float) + Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=3) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: float), _col1 (type: float), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +73895 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloat) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +73895 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cdouble is not null (type: boolean) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cdouble (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (cdouble BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(cdouble, DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cdouble + druid.fieldTypes double + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"and","fields":[{"type":"and","fields":[{"type":"bound","dimension":"cdouble","lower":"DynamicValue(RS_4_alltypesorc_small_cdouble_min)","upper":"DynamicValue(RS_4_alltypesorc_small_cdouble_max)","lowerStrict":false,"upperStrict":false,"extractionFn":null,"ordering":{"type":"numeric"}},{"type":"bloom","dimension":"cdouble","bloomKFilter":"BAAAAAgAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA","extractionFn":null}]},{"type":"not","field":{"type":"selector","dimension":"cdouble","value":null,"extractionFn":null}}]},"columns":["cdouble"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (cdouble BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(cdouble, DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: cdouble (type: double) + sort order: + + Map-reduce partition columns: cdouble (type: double) + Statistics: Num rows: 9173 Data size: 69728 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: double) + 1 cdouble (type: double) + Statistics: Num rows: 10090 Data size: 76700 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdouble) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +211968 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = druid_table_alltypesorc.`__time`) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = druid_table_alltypesorc.`__time`) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ctimestamp1 is not null (type: boolean) + Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ctimestamp1 (type: timestamp) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: CAST( _col0 AS timestamp with local time zone) (type: timestamp with local time zone) + sort order: + + Map-reduce partition columns: CAST( _col0 AS timestamp with local time zone) (type: timestamp with local time zone) + Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: CAST( _col0 AS timestamp with local time zone) (type: timestamp with local time zone) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 2760 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: timestamp with local time zone), _col1 (type: timestamp with local time zone), _col2 (type: binary) + Execution mode: llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (vc BETWEEN DynamicValue(RS_4_alltypesorc_small_ctimestamp1_min) AND DynamicValue(RS_4_alltypesorc_small_ctimestamp1_max) and in_bloom_filter(vc, DynamicValue(RS_4_alltypesorc_small_ctimestamp1_bloom_filter))) (type: boolean) + properties: + druid.fieldNames vc + druid.fieldTypes timestamp with local time zone + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[{"type":"expression","name":"vc","expression":"\"__time\"","outputType":"LONG"}],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":null,"columns":["vc"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (in_bloom_filter(vc, DynamicValue(RS_4_alltypesorc_small_ctimestamp1_bloom_filter)) and vc BETWEEN DynamicValue(RS_4_alltypesorc_small_ctimestamp1_min) AND DynamicValue(RS_4_alltypesorc_small_ctimestamp1_max)) (type: boolean) + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: vc (type: timestamp with local time zone) + sort order: + + Map-reduce partition columns: vc (type: timestamp with local time zone) + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Execution mode: llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 CAST( _col0 AS timestamp with local time zone) (type: timestamp with local time zone) + 1 vc (type: timestamp with local time zone) + Statistics: Num rows: 10090 Data size: 383504 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: timestamp with local time zone), _col1 (type: timestamp with local time zone), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp)) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +434493 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp)) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.ctimestamp1 = cast(druid_table_alltypesorc.`__time` as timestamp)) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +434493 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cboolean1 is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cboolean1 (type: boolean) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: boolean) + sort order: + + Map-reduce partition columns: _col0 (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: boolean) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: boolean), _col1 (type: boolean), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (cboolean1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cboolean1_min) AND DynamicValue(RS_4_alltypesorc_small_cboolean1_max) and in_bloom_filter(cboolean1, DynamicValue(RS_4_alltypesorc_small_cboolean1_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cboolean1 + druid.fieldTypes boolean + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cboolean1","value":null,"extractionFn":null}},"columns":["cboolean1"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (cboolean1 BETWEEN DynamicValue(RS_4_alltypesorc_small_cboolean1_min) AND DynamicValue(RS_4_alltypesorc_small_cboolean1_max) and in_bloom_filter(cboolean1, DynamicValue(RS_4_alltypesorc_small_cboolean1_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: cboolean1 (type: boolean) + sort order: + + Map-reduce partition columns: cboolean1 (type: boolean) + Statistics: Num rows: 9173 Data size: 34864 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: boolean) + 1 cboolean1 (type: boolean) + Statistics: Num rows: 10090 Data size: 38350 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: boolean), _col1 (type: boolean), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +418071 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cboolean1 = druid_table_alltypesorc.cboolean1) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +418071 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cint is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cint (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: UDFToDouble(_col0) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=69) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (UDFToDouble(cintstring) BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(UDFToDouble(cintstring), DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cintstring + druid.fieldTypes string + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cintstring","value":null,"extractionFn":null}},"columns":["cintstring"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(cintstring) BETWEEN DynamicValue(RS_4_alltypesorc_small_cint_min) AND DynamicValue(RS_4_alltypesorc_small_cint_max) and in_bloom_filter(UDFToDouble(cintstring), DynamicValue(RS_4_alltypesorc_small_cint_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(cintstring) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(cintstring) (type: double) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 UDFToDouble(cintstring) (type: double) + Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=69) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cint = druid_table_alltypesorc.cintstring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +69 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cdouble is not null (type: boolean) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cdouble (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: double) + sort order: + + Map-reduce partition columns: _col0 (type: double) + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (UDFToDouble(cdoublestring) BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(UDFToDouble(cdoublestring), DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cdoublestring + druid.fieldTypes string + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cdoublestring","value":null,"extractionFn":null}},"columns":["cdoublestring"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(cdoublestring) BETWEEN DynamicValue(RS_4_alltypesorc_small_cdouble_min) AND DynamicValue(RS_4_alltypesorc_small_cdouble_max) and in_bloom_filter(UDFToDouble(cdoublestring), DynamicValue(RS_4_alltypesorc_small_cdouble_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(cdoublestring) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(cdoublestring) (type: double) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: double) + 1 UDFToDouble(cdoublestring) (type: double) + Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cdouble = druid_table_alltypesorc.cdoublestring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 +PREHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 5 <- Reducer 4 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: alltypesorc_small + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: cfloat is not null (type: boolean) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cfloat (type: float) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: UDFToDouble(_col0) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(_col0) (type: double) + Statistics: Num rows: 69 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: UDFToDouble(_col0) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 69 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=3) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + Execution mode: vectorized, llap + LLAP IO: all inputs + Map 5 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + filterExpr: (UDFToDouble(cfloatstring) BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(UDFToDouble(cfloatstring), DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean) + properties: + druid.fieldNames cfloatstring + druid.fieldTypes string + druid.query.json {"queryType":"scan","dataSource":{"type":"table","name":"default.druid_table_alltypesorc"},"intervals":{"type":"LegacySegmentSpec","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]},"virtualColumns":[],"resultFormat":"compactedList","batchSize":20480,"limit":9223372036854775807,"filter":{"type":"not","field":{"type":"selector","dimension":"cfloatstring","value":null,"extractionFn":null}},"columns":["cfloatstring"],"legacy":null,"context":null,"descending":false,"granularity":{"type":"all"}} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(cfloatstring) BETWEEN DynamicValue(RS_4_alltypesorc_small_cfloat_min) AND DynamicValue(RS_4_alltypesorc_small_cfloat_max) and in_bloom_filter(UDFToDouble(cfloatstring), DynamicValue(RS_4_alltypesorc_small_cfloat_bloom_filter))) (type: boolean) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(cfloatstring) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(cfloatstring) (type: double) + Statistics: Num rows: 9173 Data size: 1603744 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(_col0) (type: double) + 1 UDFToDouble(cfloatstring) (type: double) + Statistics: Num rows: 10090 Data size: 1764118 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 4 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=3) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary) + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +73895 +PREHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring) +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc_small +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from alltypesorc_small join druid_table_alltypesorc on (alltypesorc_small.cfloat = druid_table_alltypesorc.cfloatstring) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc_small +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +73895 -- 2.15.2 (Apple Git-101.1)