From 0463de611ba77372396b23c4739e4618144e43f3 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 11 Sep 2018 15:59:20 -0700 Subject: [PATCH] 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 bde815ee8c..c3a33bac36 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 5b8a601131..c00884ea1c 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 f21bbee078..9674fb7ce7 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); @@ -973,7 +986,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")) { @@ -1125,7 +1139,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 f0e175057f..5cdba43cf4 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 b9eb367f0f..26321e1560 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 ecb4360623..df679f1fac 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 0125c24ce8..16aab9b95c 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 400262a107..83ea7214b0 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 425a5bbfc5..d2287fb902 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 5a6756ecbf..0000000000 --- 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 0000000000..0f605ee116 --- /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 081bc279b8..c2e17c9e80 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 a918df475d..adc9ea5960 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 1ec8b5c238..21808ed57a 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 4644df1fcc..8b520bb756 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 221081ae7d..2b963a59c0 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 ff169e88c8..4057739cd3 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 611cad38ca..c49a6e3358 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 6b6fa3d390..1bd8452fe7 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 6c6514fea0..9cd40fa081 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 1ccca5f117..3d9c5a8b43 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 946a0753ee..a3095f6e1d 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 fcda3efd34..3f82428ffe 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 e24bfff495..58398b3ef6 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 627f0787cb..9b1b89ab55 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 ac3d2a51fe..a99e5c38f3 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 f52e721763..122d161de8 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 510330d5d0..8236b04f2c 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 bb43d512c4..a0423ffcd6 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 8b6c890e07..8dd842a41f 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 cb8fa3919b..815422965a 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 e566fcf4d7..22fa21611f 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 f81a0cae6b..8480e47865 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 9c15328ecc..7c325c8758 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 ca7bda88d6..213a1f918e 100644 --- a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out +++ b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out @@ -828,7 +828,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 @@ -837,7 +837,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 @@ -855,7 +855,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 @@ -864,7 +864,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 POSTHOOK: query: explain select unix_timestamp(from_unixtime(1396681200)) from druid_table_alltypesorc limit 1 @@ -1194,7 +1194,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 POSTHOOK: 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 @@ -1549,7 +1549,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 806262d72e..67bd4fd9d8 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 @@ -187,9 +187,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 ( @@ -300,9 +300,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 @@ -649,10 +649,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 e0feb418e2..5fca67dadf 100644 --- a/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out +++ b/ql/src/test/results/clientpositive/druid/druidmini_test1.q.out @@ -67,7 +67,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 @@ -104,7 +104,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 @@ -141,7 +141,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 POSTHOOK: query: EXPLAIN SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10 @@ -174,8 +174,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 @@ -216,7 +216,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 f077fb2204..e0d5514e22 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)