diff --git common/src/java/org/apache/hadoop/hive/conf/Constants.java common/src/java/org/apache/hadoop/hive/conf/Constants.java index 61bc9df4a0..44d0717f03 100644 --- common/src/java/org/apache/hadoop/hive/conf/Constants.java +++ common/src/java/org/apache/hadoop/hive/conf/Constants.java @@ -32,8 +32,6 @@ "org.apache.hadoop.hive.druid.io.DruidOutputFormat"; public static final String DRUID_DATA_SOURCE = "druid.datasource"; public static final String DRUID_SEGMENT_GRANULARITY = "druid.segment.granularity"; - public static final String DRUID_ROLLUP = "druid.rollup"; - public static final String DRUID_QUERY_GRANULARITY = "druid.query.granularity"; public static final String DRUID_TARGET_SHARDS_PER_GRANULARITY = "druid.segment.targetShardsPerGranularity"; public static final String DRUID_TIMESTAMP_GRANULARITY_COL_NAME = "__time_granularity"; @@ -42,21 +40,6 @@ public static final String DRUID_QUERY_FIELD_NAMES = "druid.fieldNames"; public static final String DRUID_QUERY_FIELD_TYPES = "druid.fieldTypes"; public static final String DRUID_QUERY_TYPE = "druid.query.type"; - public static final String DRUID_QUERY_FETCH = "druid.query.fetch"; - public static final String DRUID_SEGMENT_DIRECTORY = "druid.storage.storageDirectory"; - public static final String DRUID_SEGMENT_INTERMEDIATE_DIRECTORY = "druid.storage.storageDirectory.intermediate"; - - public static final String DRUID_SEGMENT_VERSION = "druid.segment.version"; - public static final String DRUID_JOB_WORKING_DIRECTORY = "druid.job.workingDirectory"; - - - public static final String KAFKA_TOPIC = "kafka.topic"; - public static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; - - public static final String DRUID_KAFKA_INGESTION_PROPERTY_PREFIX = "druid.kafka.ingestion."; - public static final String DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX = DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "consumer."; - /* Kafka Ingestion state - valid values - START/STOP/RESET */ - public static final String DRUID_KAFKA_INGESTION = "druid.kafka.ingestion"; public static final String JDBC_HIVE_STORAGE_HANDLER_ID = "org.apache.hive.storage.jdbc.JdbcStorageHandler"; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java new file mode 100644 index 0000000000..2b12d8d2ac --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java @@ -0,0 +1,140 @@ +/* + * 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; + +import com.google.common.collect.ImmutableMap; +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.segment.IndexSpec; +import io.druid.segment.indexing.DataSchema; +import org.apache.hadoop.hive.druid.json.KafkaSupervisorIOConfig; +import org.apache.hadoop.hive.druid.json.KafkaSupervisorSpec; +import org.apache.hadoop.hive.druid.json.KafkaSupervisorTuningConfig; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URL; +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hadoop.hive.druid.DruidStorageHandlerUtils.JSON_MAPPER; + +/** + * Class containing some Utility methods for Kafka Ingestion. + */ +final class DruidKafkaUtils { + private static final Logger LOG = LoggerFactory.getLogger(DruidKafkaUtils.class); + private static final SessionState.LogHelper console = new SessionState.LogHelper(LOG); + + static KafkaSupervisorSpec createKafkaSupervisorSpec(Table table, + String kafkaTopic, + String kafka_servers, + DataSchema dataSchema, + IndexSpec indexSpec) { + return new KafkaSupervisorSpec(dataSchema, + new KafkaSupervisorTuningConfig(DruidStorageHandlerUtils.getIntegerProperty(table, + DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxRowsInMemory"), + DruidStorageHandlerUtils.getIntegerProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxRowsPerSegment"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "intermediatePersistPeriod"), + null, + // basePersistDirectory - use druid default, no need to be configured by user + DruidStorageHandlerUtils.getIntegerProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxPendingPersists"), + indexSpec, + null, + // buildV9Directly - use druid default, no need to be configured by user + DruidStorageHandlerUtils.getBooleanProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "reportParseExceptions"), + DruidStorageHandlerUtils.getLongProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "handoffConditionTimeout"), + DruidStorageHandlerUtils.getBooleanProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "resetOffsetAutomatically"), + DruidStorageHandlerUtils.getIntegerProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "workerThreads"), + DruidStorageHandlerUtils.getIntegerProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "chatThreads"), + DruidStorageHandlerUtils.getLongProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "chatRetries"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "httpTimeout"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "shutdownTimeout"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "offsetFetchPeriod")), + new KafkaSupervisorIOConfig(kafkaTopic, + // Mandatory Property + DruidStorageHandlerUtils.getIntegerProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "replicas"), + DruidStorageHandlerUtils.getIntegerProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "taskCount"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "taskDuration"), + getKafkaConsumerProperties(table, kafka_servers), + // Mandatory Property + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "startDelay"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "period"), + DruidStorageHandlerUtils.getBooleanProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "useEarliestOffset"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "completionTimeout"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "lateMessageRejectionPeriod"), + DruidStorageHandlerUtils.getPeriodProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "earlyMessageRejectionPeriod"), + DruidStorageHandlerUtils.getBooleanProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "skipOffsetGaps")), + new HashMap<>()); + } + + private static Map getKafkaConsumerProperties(Table table, String kafka_servers) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + builder.put(KafkaSupervisorIOConfig.BOOTSTRAP_SERVERS_KEY, kafka_servers); + for (Map.Entry entry : table.getParameters().entrySet()) { + if (entry.getKey().startsWith(DruidStorageHandlerUtils.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX)) { + String propertyName = entry.getKey().substring(DruidStorageHandlerUtils.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX.length()); + builder.put(propertyName, entry.getValue()); + } + } + return builder.build(); + } + + static void updateKafkaIngestionSpec(String overlordAddress, KafkaSupervisorSpec spec) { + try { + String task = JSON_MAPPER.writeValueAsString(spec); + console.printInfo("submitting kafka Spec {}", task); + LOG.info("submitting kafka Supervisor Spec {}", task); + FullResponseHolder + response = + DruidStorageHandlerUtils.getResponseFromCurrentLeader(DruidStorageHandler.getHttpClient(), + new Request(HttpMethod.POST, + new URL(String.format("http://%s/druid/indexer/v1/supervisor", overlordAddress))).setContent( + "application/json", + JSON_MAPPER.writeValueAsBytes(spec)), + new FullResponseHandler(Charset.forName("UTF-8"))); + if (response.getStatus().equals(HttpResponseStatus.OK)) { + String + msg = + String.format("Kafka Supervisor for [%s] Submitted Successfully to druid.", + spec.getDataSchema().getDataSource()); + LOG.info(msg); + console.printInfo(msg); + } else { + throw new IOException(String.format("Unable to update Kafka Ingestion for Druid status [%d] full response [%s]", + response.getStatus().getCode(), + response.getContent())); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + static boolean isKafkaStreamingTable(Table table) { + // For kafka Streaming tables it is mandatory to set a kafka topic. + return DruidStorageHandlerUtils.getTableProperty(table, DruidStorageHandlerUtils.KAFKA_TOPIC) != null; + } +} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java index cc38904b39..96c16f87e7 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ 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; @@ -24,7 +25,6 @@ import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import io.druid.data.input.impl.DimensionSchema; @@ -63,20 +63,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.druid.io.DruidOutputFormat; 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.KafkaSupervisorSpec; -import org.apache.hadoop.hive.druid.json.KafkaSupervisorTuningConfig; import org.apache.hadoop.hive.druid.security.KerberosHttpClient; import org.apache.hadoop.hive.druid.serde.DruidSerDe; import org.apache.hadoop.hive.metastore.DefaultHiveMetaHook; import org.apache.hadoop.hive.metastore.HiveMetaHook; -import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.EnvironmentContext; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.LockType; @@ -114,7 +112,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -127,20 +124,20 @@ /** * DruidStorageHandler provides a HiveStorageHandler implementation for Druid. */ -@SuppressWarnings({ "rawtypes" }) -public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStorageHandler { +@SuppressWarnings({ "rawtypes" }) public class DruidStorageHandler extends DefaultHiveMetaHook + implements HiveStorageHandler { - protected static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandler.class); + private static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandler.class); - protected static final SessionState.LogHelper console = new SessionState.LogHelper(LOG); + private static final SessionState.LogHelper console = new SessionState.LogHelper(LOG); public static final String SEGMENTS_DESCRIPTOR_DIR_NAME = "segmentsDescriptorDir"; - public static final String INTERMEDIATE_SEGMENT_DIR_NAME = "intermediateSegmentDir"; + private static final String INTERMEDIATE_SEGMENT_DIR_NAME = "intermediateSegmentDir"; private static final HttpClient HTTP_CLIENT; - private static List allowedAlterTypes = ImmutableList.of("ADDPROPS", "DROPPROPS", "ADDCOLS"); + private static final List allowedAlterTypes = ImmutableList.of("ADDPROPS", "DROPPROPS", "ADDCOLS"); static { final Lifecycle lifecycle = new Lifecycle(); @@ -150,7 +147,7 @@ LOG.error("Issues with lifecycle start", e); } HTTP_CLIENT = makeHttpClient(lifecycle); - ShutdownHookManager.addShutdownHook(() -> lifecycle.stop()); + ShutdownHookManager.addShutdownHook(lifecycle::stop); } private SQLMetadataConnector connector; @@ -166,55 +163,42 @@ public DruidStorageHandler() { } - @VisibleForTesting - public DruidStorageHandler(SQLMetadataConnector connector, - MetadataStorageTablesConfig druidMetadataStorageTablesConfig - ) { + @VisibleForTesting public DruidStorageHandler(SQLMetadataConnector connector, + MetadataStorageTablesConfig druidMetadataStorageTablesConfig) { this.connector = connector; this.druidMetadataStorageTablesConfig = druidMetadataStorageTablesConfig; } - @Override - public Class getInputFormatClass() { + @Override public Class getInputFormatClass() { return DruidQueryBasedInputFormat.class; } - @Override - public Class getOutputFormatClass() { + @Override public Class getOutputFormatClass() { return DruidOutputFormat.class; } - @Override - public Class getSerDeClass() { + @Override public Class getSerDeClass() { return DruidSerDe.class; } - @Override - public HiveMetaHook getMetaHook() { + @Override public HiveMetaHook getMetaHook() { return this; } - @Override - public HiveAuthorizationProvider getAuthorizationProvider() { + @Override public HiveAuthorizationProvider getAuthorizationProvider() { return new DefaultHiveAuthorizationProvider(); } - @Override - public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties - ) { + @Override public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { } - @Override - public void configureInputJobCredentials(TableDesc tableDesc, Map jobSecrets - ) { + @Override public void configureInputJobCredentials(TableDesc tableDesc, Map jobSecrets) { } - @Override - public void preCreateTable(Table table) throws MetaException { - if(!StringUtils - .isEmpty(table.getSd().getLocation())) { + @Override public void preCreateTable(Table table) throws MetaException { + if (!StringUtils.isEmpty(table.getSd().getLocation())) { throw new MetaException("LOCATION may not be specified for Druid"); } if (table.getPartitionKeysSize() != 0) { @@ -224,13 +208,13 @@ public void preCreateTable(Table table) throws MetaException { throw new MetaException("CLUSTERED BY may not be specified for Druid"); } String dataSourceName = table.getParameters().get(Constants.DRUID_DATA_SOURCE); - if(dataSourceName != null){ + if (dataSourceName != null) { // Already Existing datasource in Druid. return; } // create dataSourceName based on Hive Table name - dataSourceName = Warehouse.getQualifiedName(table); + dataSourceName = TableName.getDbTable(table.getDbName(), table.getTableName()); try { // NOTE: This just created druid_segments table in Druid metastore. // This is needed for the case when hive is started before any of druid services @@ -240,8 +224,9 @@ public void preCreateTable(Table table) throws MetaException { LOG.error("Exception while trying to create druid segments table", e); throw new MetaException(e.getMessage()); } - Collection existingDataSources = DruidStorageHandlerUtils - .getAllDataSourceNames(getConnector(), getDruidMetadataStorageTablesConfig()); + Collection + existingDataSources = + DruidStorageHandlerUtils.getAllDataSourceNames(getConnector(), getDruidMetadataStorageTablesConfig()); LOG.debug("pre-create data source with name {}", dataSourceName); // Check for existence of for the datasource we are going to create in druid_segments table. if (existingDataSources.contains(dataSourceName)) { @@ -250,14 +235,12 @@ public void preCreateTable(Table table) throws MetaException { table.getParameters().put(Constants.DRUID_DATA_SOURCE, dataSourceName); } - @Override - public void rollbackCreateTable(Table table) { + @Override public void rollbackCreateTable(Table table) { cleanWorkingDir(); } - @Override - public void commitCreateTable(Table table) throws MetaException { - if(isKafkaStreamingTable(table)){ + @Override public void commitCreateTable(Table table) throws MetaException { + if (DruidKafkaUtils.isKafkaStreamingTable(table)) { updateKafkaIngestion(table); } // For CTAS queries when user has explicitly specified the datasource. @@ -265,14 +248,21 @@ public void commitCreateTable(Table table) throws MetaException { this.commitInsertTable(table, false); } - private void updateKafkaIngestion(Table table){ - final String overlordAddress = HiveConf - .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS); + private void updateKafkaIngestion(Table table) { + final String overlordAddress = HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS); - final String dataSourceName = Preconditions.checkNotNull(getTableProperty(table, Constants.DRUID_DATA_SOURCE), "Druid datasource name is null"); + final String + dataSourceName = + Preconditions.checkNotNull(DruidStorageHandlerUtils.getTableProperty(table, Constants.DRUID_DATA_SOURCE), + "Druid datasource name is null"); - final String kafkaTopic = Preconditions.checkNotNull(getTableProperty(table, Constants.KAFKA_TOPIC), "kafka topic is null"); - final String kafka_servers = Preconditions.checkNotNull(getTableProperty(table, Constants.KAFKA_BOOTSTRAP_SERVERS), "kafka connect string is null"); + final String + kafkaTopic = + Preconditions.checkNotNull(DruidStorageHandlerUtils.getTableProperty(table, DruidStorageHandlerUtils.KAFKA_TOPIC), "kafka topic is null"); + final String + kafka_servers = + Preconditions.checkNotNull(DruidStorageHandlerUtils.getTableProperty(table, DruidStorageHandlerUtils.KAFKA_BOOTSTRAP_SERVERS), + "kafka connect string is null"); Properties tableProperties = new Properties(); tableProperties.putAll(table.getParameters()); @@ -283,175 +273,94 @@ private void updateKafkaIngestion(Table table){ List columnNames = new ArrayList<>(columns.size()); List columnTypes = new ArrayList<>(columns.size()); - for(FieldSchema schema: columns) { + for (FieldSchema schema : columns) { columnNames.add(schema.getName()); columnTypes.add(TypeInfoUtils.getTypeInfoFromTypeString(schema.getType())); } - Pair, AggregatorFactory[]> dimensionsAndAggregates = DruidStorageHandlerUtils - .getDimensionsAndAggregates(getConf(), columnNames, columnTypes); + Pair, AggregatorFactory[]> + dimensionsAndAggregates = + DruidStorageHandlerUtils.getDimensionsAndAggregates(columnNames, columnTypes); if (!columnNames.contains(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN)) { - throw new IllegalStateException( - "Timestamp column (' " + DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN + - "') not specified in create table; list of columns is : " + - columnNames); + throw new IllegalStateException("Timestamp column (' " + + DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN + + "') not specified in create table; list of columns is : " + + columnNames); } - final InputRowParser inputRowParser = new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, "auto", null), - new DimensionsSpec(dimensionsAndAggregates.lhs, null, null), + final InputRowParser + inputRowParser = + new StringInputRowParser(new JSONParseSpec(new TimestampSpec(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, + "auto", + null), new DimensionsSpec(dimensionsAndAggregates.lhs, null, null), null, null), "UTF-8"); + + final Map + inputParser = + JSON_MAPPER.convertValue(inputRowParser, new TypeReference>() { + }); + final DataSchema + dataSchema = + new DataSchema(dataSourceName, + inputParser, + dimensionsAndAggregates.rhs, + granularitySpec, null, - null - ), "UTF-8"); - - Map inputParser = JSON_MAPPER - .convertValue(inputRowParser, Map.class); - final DataSchema dataSchema = new DataSchema( - dataSourceName, - inputParser, - dimensionsAndAggregates.rhs, - granularitySpec, - null, - DruidStorageHandlerUtils.JSON_MAPPER - ); + DruidStorageHandlerUtils.JSON_MAPPER); IndexSpec indexSpec = DruidStorageHandlerUtils.getIndexSpec(getConf()); - KafkaSupervisorSpec spec = createKafkaSupervisorSpec(table, kafkaTopic, kafka_servers, - dataSchema, indexSpec); + KafkaSupervisorSpec spec = DruidKafkaUtils.createKafkaSupervisorSpec(table, kafkaTopic, kafka_servers, dataSchema, indexSpec); // Fetch existing Ingestion Spec from Druid, if any KafkaSupervisorSpec existingSpec = fetchKafkaIngestionSpec(table); - String targetState = getTableProperty(table, Constants.DRUID_KAFKA_INGESTION); - if(targetState == null){ + String targetState = DruidStorageHandlerUtils.getTableProperty(table, DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION); + if (targetState == null) { // Case when user has not specified any ingestion state in the current command // if there is a kafka supervisor running then keep it last known state is START otherwise STOP. targetState = existingSpec == null ? "STOP" : "START"; } - if(targetState.equalsIgnoreCase("STOP")){ - if(existingSpec != null){ + if (targetState.equalsIgnoreCase("STOP")) { + if (existingSpec != null) { stopKafkaIngestion(overlordAddress, dataSourceName); } - } else if(targetState.equalsIgnoreCase("START")){ - if(existingSpec == null || !existingSpec.equals(spec)){ - updateKafkaIngestionSpec(overlordAddress, spec); + } else if (targetState.equalsIgnoreCase("START")) { + if (existingSpec == null || !existingSpec.equals(spec)) { + DruidKafkaUtils.updateKafkaIngestionSpec(overlordAddress, spec); } - } else if(targetState.equalsIgnoreCase("RESET")){ + } else if (targetState.equalsIgnoreCase("RESET")) { // Case when there are changes in multiple table properties. - if(existingSpec != null && !existingSpec.equals(spec)){ - updateKafkaIngestionSpec(overlordAddress, spec); + if (existingSpec != null && !existingSpec.equals(spec)) { + DruidKafkaUtils.updateKafkaIngestionSpec(overlordAddress, spec); } resetKafkaIngestion(overlordAddress, dataSourceName); } else { - throw new IllegalArgumentException(String.format("Invalid value for property [%s], Valid values are [START, STOP, RESET]", Constants.DRUID_KAFKA_INGESTION)); + throw new IllegalArgumentException(String.format( + "Invalid value for property [%s], Valid values are [START, STOP, RESET]", + DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION)); } // We do not want to keep state in two separate places so remove from hive table properties. - table.getParameters().remove(Constants.DRUID_KAFKA_INGESTION); - } - - private static KafkaSupervisorSpec createKafkaSupervisorSpec(Table table, String kafkaTopic, - String kafka_servers, DataSchema dataSchema, IndexSpec indexSpec) { - return new KafkaSupervisorSpec(dataSchema, - new KafkaSupervisorTuningConfig( - getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxRowsInMemory"), - getIntegerProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "maxRowsPerSegment"), - 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"), - indexSpec, - null, // buildV9Directly - use druid default, no need to be configured by user - 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"), - 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 - 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"), - getKafkaConsumerProperties(table, kafka_servers), // Mandatory Property - getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "startDelay"), - getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "period"), - getBooleanProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "useEarliestOffset"), - getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "completionTimeout"), - getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "lateMessageRejectionPeriod"), - getPeriodProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "earlyMessageRejectionPeriod"), - getBooleanProperty(table, Constants.DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "skipOffsetGaps")), - new HashMap() - ); - } - - private static Map getKafkaConsumerProperties(Table table, String kafka_servers) { - ImmutableMap.Builder builder = ImmutableMap.builder(); - builder.put(KafkaSupervisorIOConfig.BOOTSTRAP_SERVERS_KEY, kafka_servers); - for (Map.Entry entry : table.getParameters().entrySet()) { - if (entry.getKey().startsWith(Constants.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX)) { - String propertyName = entry.getKey() - .substring(Constants.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX.length()); - builder.put(propertyName, entry.getValue()); - } - } - return builder.build(); - } - - private static void updateKafkaIngestionSpec(String overlordAddress, KafkaSupervisorSpec spec) { - try { - String task = JSON_MAPPER.writeValueAsString(spec); - console.printInfo("submitting kafka Spec {}", task); - LOG.info("submitting kafka Supervisor Spec {}", task); - FullResponseHolder response = DruidStorageHandlerUtils - .getResponseFromCurrentLeader(getHttpClient(), new Request(HttpMethod.POST, - new URL(String - .format("http://%s/druid/indexer/v1/supervisor", overlordAddress)) - ) - .setContent( - "application/json", - JSON_MAPPER.writeValueAsBytes(spec) - ), new FullResponseHandler( - Charset.forName("UTF-8"))); - if (response.getStatus().equals(HttpResponseStatus.OK)) { - String msg = String.format("Kafka Supervisor for [%s] Submitted Successfully to druid.", spec.getDataSchema().getDataSource()); - LOG.info(msg); - console.printInfo(msg); - } else { - throw new IOException(String - .format("Unable to update Kafka Ingestion for Druid status [%d] full response [%s]", - response.getStatus().getCode(), response.getContent())); - } - } catch (Exception e) { - throw new RuntimeException(e); - } + table.getParameters().remove(DruidStorageHandlerUtils.DRUID_KAFKA_INGESTION); } private void resetKafkaIngestion(String overlordAddress, String dataSourceName) { try { - FullResponseHolder response = RetryUtils - .retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader( - getHttpClient(), - new Request(HttpMethod.POST, - new URL(String - .format("http://%s/druid/indexer/v1/supervisor/%s/reset", - overlordAddress, - dataSourceName - )) - ), new FullResponseHandler( - Charset.forName("UTF-8")) - ), - input -> input instanceof IOException, - getMaxRetryCount() - ); - if(response.getStatus().equals(HttpResponseStatus.OK)) { + FullResponseHolder + response = + RetryUtils.retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader(getHttpClient(), + new Request(HttpMethod.POST, + new URL(String.format("http://%s/druid/indexer/v1/supervisor/%s/reset", + overlordAddress, + dataSourceName))), + new FullResponseHandler(Charset.forName("UTF-8"))), + input -> input instanceof IOException, + getMaxRetryCount()); + if (response.getStatus().equals(HttpResponseStatus.OK)) { console.printInfo("Druid Kafka Ingestion Reset successful."); } else { - throw new IOException(String - .format("Unable to reset Kafka Ingestion Druid status [%d] full response [%s]", - response.getStatus().getCode(), response.getContent())); + throw new IOException(String.format("Unable to reset Kafka Ingestion Druid status [%d] full response [%s]", + response.getStatus().getCode(), + response.getContent())); } } catch (Exception e) { throw new RuntimeException(e); @@ -460,27 +369,22 @@ private void resetKafkaIngestion(String overlordAddress, String dataSourceName) private void stopKafkaIngestion(String overlordAddress, String dataSourceName) { try { - FullResponseHolder response = RetryUtils - .retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader( - getHttpClient(), - new Request(HttpMethod.POST, - new URL(String - .format("http://%s/druid/indexer/v1/supervisor/%s/shutdown", - overlordAddress, - dataSourceName - )) - ), new FullResponseHandler( - Charset.forName("UTF-8")) - ), - input -> input instanceof IOException, - getMaxRetryCount() - ); + FullResponseHolder + response = + RetryUtils.retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader(getHttpClient(), + new Request(HttpMethod.POST, + new URL(String.format("http://%s/druid/indexer/v1/supervisor/%s/shutdown", + overlordAddress, + dataSourceName))), + new FullResponseHandler(Charset.forName("UTF-8"))), + input -> input instanceof IOException, + getMaxRetryCount()); if (response.getStatus().equals(HttpResponseStatus.OK)) { console.printInfo("Druid Kafka Ingestion shutdown successful."); } else { - throw new IOException(String - .format("Unable to stop Kafka Ingestion Druid status [%d] full response [%s]", - response.getStatus().getCode(), response.getContent())); + throw new IOException(String.format("Unable to stop Kafka Ingestion Druid status [%d] full response [%s]", + response.getStatus().getCode(), + response.getContent())); } } catch (Exception e) { throw new RuntimeException(e); @@ -490,39 +394,35 @@ private void stopKafkaIngestion(String overlordAddress, String dataSourceName) { private KafkaSupervisorSpec fetchKafkaIngestionSpec(Table table) { // Stop Kafka Ingestion first - final String overlordAddress = Preconditions.checkNotNull(HiveConf - .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), - "Druid Overlord Address is null"); - String dataSourceName = Preconditions - .checkNotNull(getTableProperty(table, Constants.DRUID_DATA_SOURCE), + final String + overlordAddress = + Preconditions.checkNotNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), + "Druid Overlord Address is null"); + String + dataSourceName = + Preconditions.checkNotNull(DruidStorageHandlerUtils.getTableProperty(table, Constants.DRUID_DATA_SOURCE), "Druid Datasource name is null"); try { - FullResponseHolder response = RetryUtils - .retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader( - getHttpClient(), - new Request(HttpMethod.GET, - new URL(String - .format("http://%s/druid/indexer/v1/supervisor/%s", - overlordAddress, - dataSourceName - )) - ), new FullResponseHandler( - Charset.forName("UTF-8")) - ), - input -> input instanceof IOException, - getMaxRetryCount() - ); - if(response.getStatus().equals(HttpResponseStatus.OK)) { - return JSON_MAPPER - .readValue(response.getContent(), KafkaSupervisorSpec.class); + FullResponseHolder + response = + RetryUtils.retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader(getHttpClient(), + new Request(HttpMethod.GET, + new URL(String.format("http://%s/druid/indexer/v1/supervisor/%s", overlordAddress, dataSourceName))), + new FullResponseHandler(Charset.forName("UTF-8"))), + input -> input instanceof IOException, + getMaxRetryCount()); + if (response.getStatus().equals(HttpResponseStatus.OK)) { + return JSON_MAPPER.readValue(response.getContent(), KafkaSupervisorSpec.class); // Druid Returns 400 Bad Request when not found. - } else if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) || response.getStatus().equals(HttpResponseStatus.BAD_REQUEST)) { + } else if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) || response.getStatus() + .equals(HttpResponseStatus.BAD_REQUEST)) { LOG.debug("No Kafka Supervisor found for datasource[%s]", dataSourceName); return null; } else { - throw new IOException(String - .format("Unable to fetch Kafka Ingestion Spec from Druid status [%d] full response [%s]", - response.getStatus().getCode(), response.getContent())); + throw new IOException(String.format( + "Unable to fetch Kafka Ingestion Spec from Druid status [%d] full response [%s]", + response.getStatus().getCode(), + response.getContent())); } } catch (Exception e) { throw new RuntimeException("Exception while fetching kafka ingestion spec from druid", e); @@ -530,41 +430,42 @@ private KafkaSupervisorSpec fetchKafkaIngestionSpec(Table table) { } /** - * Fetches kafka supervisor status report from druid overlod. - * @param table + * Fetches kafka supervisor status report from druid overlord. This method will return null if can not fetch report + * + * @param table object. * @return kafka supervisor report or null when druid overlord is unreachable. */ - @Nullable - private KafkaSupervisorReport fetchKafkaSupervisorReport(Table table) { - final String overlordAddress = Preconditions.checkNotNull(HiveConf - .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), + @Nullable private KafkaSupervisorReport fetchKafkaSupervisorReport(Table table) { + final String + overlordAddress = + Preconditions.checkNotNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), "Druid Overlord Address is null"); - String dataSourceName = Preconditions - .checkNotNull(getTableProperty(table, Constants.DRUID_DATA_SOURCE), - "Druid Datasource name is null"); + final String + dataSourceName = + Preconditions.checkNotNull(DruidStorageHandlerUtils.getTableProperty(table, Constants.DRUID_DATA_SOURCE), + "Druid Datasource name is null"); try { - FullResponseHolder response = RetryUtils.retry(() -> DruidStorageHandlerUtils - .getResponseFromCurrentLeader(getHttpClient(), new Request(HttpMethod.GET, - new URL(String - .format("http://%s/druid/indexer/v1/supervisor/%s/status", - overlordAddress, - dataSourceName - )) - ), new FullResponseHandler( - Charset.forName("UTF-8"))), + FullResponseHolder + response = + RetryUtils.retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader(getHttpClient(), + new Request(HttpMethod.GET, + new URL(String.format("http://%s/druid/indexer/v1/supervisor/%s/status", + overlordAddress, + dataSourceName))), + new FullResponseHandler(Charset.forName("UTF-8"))), input -> input instanceof IOException, - getMaxRetryCount() - ); + getMaxRetryCount()); if (response.getStatus().equals(HttpResponseStatus.OK)) { - return DruidStorageHandlerUtils.JSON_MAPPER - .readValue(response.getContent(), KafkaSupervisorReport.class); + return DruidStorageHandlerUtils.JSON_MAPPER.readValue(response.getContent(), KafkaSupervisorReport.class); // Druid Returns 400 Bad Request when not found. - } else if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) || response.getStatus().equals(HttpResponseStatus.BAD_REQUEST)) { + } else if (response.getStatus().equals(HttpResponseStatus.NOT_FOUND) || response.getStatus() + .equals(HttpResponseStatus.BAD_REQUEST)) { LOG.info("No Kafka Supervisor found for datasource[%s]", dataSourceName); return null; } else { LOG.error("Unable to fetch Kafka Supervisor status [%d] full response [%s]", - response.getStatus().getCode(), response.getContent()); + response.getStatus().getCode(), + response.getContent()); return null; } } catch (Exception e) { @@ -577,90 +478,73 @@ private KafkaSupervisorReport fetchKafkaSupervisorReport(Table table) { * Creates metadata moves then commit the Segment's metadata to Druid metadata store in one TxN * * @param table Hive table - * @param overwrite true if it is an insert overwrite table - * - * @throws MetaException if errors occurs. + * @param overwrite true if it is an insert overwrite table. */ - protected List loadAndCommitDruidSegments(Table table, boolean overwrite, List segmentsToLoad) + private List loadAndCommitDruidSegments(Table table, boolean overwrite, List segmentsToLoad) throws IOException, CallbackFailedException { final String dataSourceName = table.getParameters().get(Constants.DRUID_DATA_SOURCE); - final String segmentDirectory = - table.getParameters().get(Constants.DRUID_SEGMENT_DIRECTORY) != null - ? table.getParameters().get(Constants.DRUID_SEGMENT_DIRECTORY) - : HiveConf.getVar(getConf(), HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY); - - final HdfsDataSegmentPusherConfig hdfsSegmentPusherConfig = new HdfsDataSegmentPusherConfig(); - List publishedDataSegmentList; - - LOG.info(String.format( - "Moving [%s] Druid segments from staging directory [%s] to Deep storage [%s]", - segmentsToLoad.size(), - getStagingWorkingDir().toString(), - segmentDirectory - )); - hdfsSegmentPusherConfig.setStorageDirectory(segmentDirectory); - DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(hdfsSegmentPusherConfig, - getConf(), - JSON_MAPPER - ); - publishedDataSegmentList = DruidStorageHandlerUtils.publishSegmentsAndCommit( - getConnector(), - getDruidMetadataStorageTablesConfig(), - dataSourceName, - segmentsToLoad, - overwrite, - getConf(), - dataSegmentPusher - ); - return publishedDataSegmentList; + final String + segmentDirectory = + table.getParameters().get(DruidStorageHandlerUtils.DRUID_SEGMENT_DIRECTORY) != null ? + table.getParameters().get(DruidStorageHandlerUtils.DRUID_SEGMENT_DIRECTORY) : + HiveConf.getVar(getConf(), HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY); + + final HdfsDataSegmentPusherConfig hdfsSegmentPusherConfig = new HdfsDataSegmentPusherConfig(); + List publishedDataSegmentList; + + LOG.info(String.format("Moving [%s] Druid segments from staging directory [%s] to Deep storage [%s]", + segmentsToLoad.size(), + getStagingWorkingDir().toString(), + segmentDirectory)); + hdfsSegmentPusherConfig.setStorageDirectory(segmentDirectory); + DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(hdfsSegmentPusherConfig, getConf(), JSON_MAPPER); + publishedDataSegmentList = + DruidStorageHandlerUtils.publishSegmentsAndCommit(getConnector(), + getDruidMetadataStorageTablesConfig(), + dataSourceName, + segmentsToLoad, + overwrite, + getConf(), + dataSegmentPusher); + return publishedDataSegmentList; } /** * This function checks the load status of Druid segments by polling druid coordinator. * @param segments List of druid segments to check for - * - * @return count of yet to load segments. */ - private int checkLoadStatus(List segments){ - final String coordinatorAddress = HiveConf - .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_COORDINATOR_DEFAULT_ADDRESS); + private void checkLoadStatus(List segments) { + final String + coordinatorAddress = + HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_COORDINATOR_DEFAULT_ADDRESS); int maxTries = getMaxRetryCount(); - if (maxTries == 0) { - return segments.size(); - } + LOG.debug("checking load status from coordinator {}", coordinatorAddress); String coordinatorResponse; try { - coordinatorResponse = RetryUtils - .retry(() -> DruidStorageHandlerUtils - .getResponseFromCurrentLeader(getHttpClient(), new Request(HttpMethod.GET, - new URL(String.format("http://%s/status", coordinatorAddress)) - ), - new FullResponseHandler(Charset.forName("UTF-8")) - ).getContent(), - input -> input instanceof IOException, maxTries - ); + coordinatorResponse = + RetryUtils.retry(() -> DruidStorageHandlerUtils.getResponseFromCurrentLeader(getHttpClient(), + new Request(HttpMethod.GET, new URL(String.format("http://%s/status", coordinatorAddress))), + new FullResponseHandler(Charset.forName("UTF-8"))).getContent(), + input -> input instanceof IOException, + maxTries); } catch (Exception e) { - console.printInfo( - "Will skip waiting for data loading, coordinator unavailable"); - return segments.size(); + console.printInfo("Will skip waiting for data loading, coordinator unavailable"); + return; } if (Strings.isNullOrEmpty(coordinatorResponse)) { - console.printInfo( - "Will skip waiting for data loading empty response from coordinator"); - return segments.size(); + console.printInfo("Will skip waiting for data loading empty response from coordinator"); } - console.printInfo( - String.format("Waiting for the loading of [%s] segments", segments.size())); - long passiveWaitTimeMs = HiveConf - .getLongVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_PASSIVE_WAIT_TIME); + console.printInfo(String.format("Waiting for the loading of [%s] segments", segments.size())); + long passiveWaitTimeMs = HiveConf.getLongVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_PASSIVE_WAIT_TIME); Set UrlsOfUnloadedSegments = segments.stream().map(dataSegment -> { try { //Need to make sure that we are using segment identifier return new URL(String.format("http://%s/druid/coordinator/v1/datasources/%s/segments/%s", - coordinatorAddress, dataSegment.getDataSource(), dataSegment.getIdentifier() - )); + coordinatorAddress, + dataSegment.getDataSource(), + dataSegment.getIdentifier())); } catch (MalformedURLException e) { Throwables.propagate(e); } @@ -671,10 +555,11 @@ private int checkLoadStatus(List segments){ while (numRetries++ < maxTries && !UrlsOfUnloadedSegments.isEmpty()) { UrlsOfUnloadedSegments = ImmutableSet.copyOf(Sets.filter(UrlsOfUnloadedSegments, input -> { try { - String result = DruidStorageHandlerUtils - .getResponseFromCurrentLeader(getHttpClient(), new Request(HttpMethod.GET, input), - new FullResponseHandler(Charset.forName("UTF-8")) - ).getContent(); + String + result = + DruidStorageHandlerUtils.getResponseFromCurrentLeader(getHttpClient(), + new Request(HttpMethod.GET, input), + new FullResponseHandler(Charset.forName("UTF-8"))).getContent(); LOG.debug("Checking segment [{}] response is [{}]", input, result); return Strings.isNullOrEmpty(result); @@ -689,22 +574,18 @@ private int checkLoadStatus(List segments){ Thread.sleep(passiveWaitTimeMs); } } catch (InterruptedException e) { - Thread.interrupted(); - Throwables.propagate(e); + throw new RuntimeException(e); } } if (!UrlsOfUnloadedSegments.isEmpty()) { // We are not Throwing an exception since it might be a transient issue that is blocking loading - console.printError(String.format( - "Wait time exhausted and we have [%s] out of [%s] segments not loaded yet", - UrlsOfUnloadedSegments.size(), segments.size() - )); + console.printError(String.format("Wait time exhausted and we have [%s] out of [%s] segments not loaded yet", + UrlsOfUnloadedSegments.size(), + segments.size())); } - return UrlsOfUnloadedSegments.size(); } - @VisibleForTesting - protected void deleteSegment(DataSegment segment) throws SegmentLoadingException { + @VisibleForTesting void deleteSegment(DataSegment segment) throws SegmentLoadingException { final Path path = DruidStorageHandlerUtils.getPath(segment); LOG.info("removing segment {}, located at path {}", segment.getIdentifier(), path); @@ -722,10 +603,8 @@ protected void deleteSegment(DataSegment segment) throws SegmentLoadingException // path format -- > .../dataSource/interval/version/partitionNum/xxx.zip Path partitionNumDir = path.getParent(); if (!fs.delete(partitionNumDir, true)) { - throw new SegmentLoadingException( - "Unable to kill segment, failed to delete dir [%s]", - partitionNumDir.toString() - ); + throw new SegmentLoadingException("Unable to kill segment, failed to delete dir [%s]", + partitionNumDir.toString()); } //try to delete other directories if possible @@ -753,40 +632,41 @@ private static boolean safeNonRecursiveDelete(FileSystem fs, Path path) { } } - @Override - public void preDropTable(Table table) { + @Override public void preDropTable(Table table) { // Nothing to do } - @Override - public void rollbackDropTable(Table table) { + @Override public void rollbackDropTable(Table table) { // Nothing to do } - @Override - public void commitDropTable(Table table, boolean deleteData) { - if(isKafkaStreamingTable(table)) { + @Override public void commitDropTable(Table table, boolean deleteData) { + if (DruidKafkaUtils.isKafkaStreamingTable(table)) { // Stop Kafka Ingestion first - final String overlordAddress = Preconditions.checkNotNull(HiveConf - .getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), - "Druid Overlord Address is null"); - String dataSourceName = Preconditions - .checkNotNull(getTableProperty(table, Constants.DRUID_DATA_SOURCE), + final String + overlordAddress = + Preconditions.checkNotNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_OVERLORD_DEFAULT_ADDRESS), + "Druid Overlord Address is null"); + String + dataSourceName = + Preconditions.checkNotNull(DruidStorageHandlerUtils.getTableProperty(table, Constants.DRUID_DATA_SOURCE), "Druid Datasource name is null"); stopKafkaIngestion(overlordAddress, dataSourceName); } - String dataSourceName = Preconditions - .checkNotNull(table.getParameters().get(Constants.DRUID_DATA_SOURCE), - "DataSource name is null !" - ); + String + dataSourceName = + Preconditions.checkNotNull(table.getParameters().get(Constants.DRUID_DATA_SOURCE), "DataSource name is null !"); // TODO: Move MetaStoreUtils.isExternalTablePurge(table) calls to a common place for all StorageHandlers // deleteData flag passed down to StorageHandler should be true only if // MetaStoreUtils.isExternalTablePurge(table) returns true. - if (deleteData == true && MetaStoreUtils.isExternalTablePurge(table)) { + if (deleteData && MetaStoreUtils.isExternalTablePurge(table)) { LOG.info("Dropping with purge all the data for data source {}", dataSourceName); - List dataSegmentList = DruidStorageHandlerUtils - .getDataSegmentList(getConnector(), getDruidMetadataStorageTablesConfig(), dataSourceName); + List + dataSegmentList = + DruidStorageHandlerUtils.getDataSegmentList(getConnector(), + getDruidMetadataStorageTablesConfig(), + dataSourceName); if (dataSegmentList.isEmpty()) { LOG.info("Nothing to delete for data source {}", dataSourceName); return; @@ -799,16 +679,15 @@ public void commitDropTable(Table table, boolean deleteData) { } } } - if (DruidStorageHandlerUtils - .disableDataSource(getConnector(), getDruidMetadataStorageTablesConfig(), dataSourceName)) { + if (DruidStorageHandlerUtils.disableDataSource(getConnector(), + getDruidMetadataStorageTablesConfig(), + dataSourceName)) { LOG.info("Successfully dropped druid data source {}", dataSourceName); } } - @Override - public void commitInsertTable(Table table, boolean overwrite) throws MetaException { - LOG.debug("commit insert into table {} overwrite {}", table.getTableName(), - overwrite); + @Override public void commitInsertTable(Table table, boolean overwrite) throws MetaException { + LOG.debug("commit insert into table {} overwrite {}", table.getTableName(), overwrite); try { // Check if there segments to load final Path segmentDescriptorDir = getSegmentDescriptorDir(); @@ -818,11 +697,9 @@ public void commitInsertTable(Table table, boolean overwrite) throws MetaExcepti if (segmentsToLoad.isEmpty() && overwrite) { //disable datasource //Case it is an insert overwrite we have to disable the existing Druid DataSource - DruidStorageHandlerUtils - .disableDataSource(getConnector(), getDruidMetadataStorageTablesConfig(), - dataSourceName - ); - return; + DruidStorageHandlerUtils.disableDataSource(getConnector(), + getDruidMetadataStorageTablesConfig(), + dataSourceName); } else if (!segmentsToLoad.isEmpty()) { // at this point we have Druid segments from reducers but we need to atomically // rename and commit to metadata @@ -841,48 +718,41 @@ public void commitInsertTable(Table table, boolean overwrite) throws MetaExcepti private List fetchSegmentsMetadata(Path segmentDescriptorDir) throws IOException { if (!segmentDescriptorDir.getFileSystem(getConf()).exists(segmentDescriptorDir)) { - LOG.info( - "Directory {} does not exist, ignore this if it is create statement or inserts of 0 rows," + LOG.info("Directory {} does not exist, ignore this if it is create statement or inserts of 0 rows," + " no Druid segments to move, cleaning working directory {}", - segmentDescriptorDir.toString(), getStagingWorkingDir().toString() - ); - return Collections.EMPTY_LIST; + segmentDescriptorDir.toString(), + getStagingWorkingDir().toString()); + return Collections.emptyList(); } return DruidStorageHandlerUtils.getCreatedSegments(segmentDescriptorDir, getConf()); } - @Override - public void preInsertTable(Table table, boolean overwrite) { + @Override public void preInsertTable(Table table, boolean overwrite) { } - @Override - public void rollbackInsertTable(Table table, boolean overwrite) { + @Override public void rollbackInsertTable(Table table, boolean overwrite) { // do nothing } - @Override - public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { + @Override public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { jobProperties.put(Constants.DRUID_DATA_SOURCE, tableDesc.getTableName()); - jobProperties.put(Constants.DRUID_SEGMENT_VERSION, new DateTime().toString()); - jobProperties.put(Constants.DRUID_JOB_WORKING_DIRECTORY, getStagingWorkingDir().toString()); + jobProperties.put(DruidStorageHandlerUtils.DRUID_SEGMENT_VERSION, new DateTime().toString()); + jobProperties.put(DruidStorageHandlerUtils.DRUID_JOB_WORKING_DIRECTORY, getStagingWorkingDir().toString()); // DruidOutputFormat will write segments in an intermediate directory - jobProperties.put(Constants.DRUID_SEGMENT_INTERMEDIATE_DIRECTORY, - getIntermediateSegmentDir().toString()); + jobProperties.put(DruidStorageHandlerUtils.DRUID_SEGMENT_INTERMEDIATE_DIRECTORY, getIntermediateSegmentDir().toString()); } - @Override - public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties) { + @Override public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties) { } - @Override - public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { + @Override public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { if (UserGroupInformation.isSecurityEnabled()) { // AM can not do Kerberos Auth so will do the input split generation in the HS2 - LOG.debug("Setting {} to {} to enable split generation on HS2", HiveConf.ConfVars.HIVE_AM_SPLIT_GENERATION.toString(), - Boolean.FALSE.toString() - ); + LOG.debug("Setting {} to {} to enable split generation on HS2", + HiveConf.ConfVars.HIVE_AM_SPLIT_GENERATION.toString(), + Boolean.FALSE.toString()); jobConf.set(HiveConf.ConfVars.HIVE_AM_SPLIT_GENERATION.toString(), Boolean.FALSE.toString()); } try { @@ -892,35 +762,30 @@ public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { } } - @Override - public void setConf(Configuration conf) { + @Override public void setConf(Configuration conf) { this.conf = conf; } - @Override - public Configuration getConf() { + @Override public Configuration getConf() { return conf; } - @Override public LockType getLockType(WriteEntity writeEntity - ) { + @Override public LockType getLockType(WriteEntity writeEntity) { if (writeEntity.getWriteType().equals(WriteEntity.WriteType.INSERT)) { return LockType.SHARED_READ; } return LockType.SHARED_WRITE; } - @Override - public String toString() { + @Override public String toString() { return Constants.DRUID_HIVE_STORAGE_HANDLER_ID; } - public String getUniqueId() { + private String getUniqueId() { if (uniqueId == null) { - uniqueId = Preconditions.checkNotNull( - Strings.emptyToNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVEQUERYID)), - "Hive query id is null" - ); + uniqueId = + Preconditions.checkNotNull(Strings.emptyToNull(HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVEQUERYID)), + "Hive query id is null"); } return uniqueId; } @@ -933,8 +798,7 @@ private MetadataStorageTablesConfig getDruidMetadataStorageTablesConfig() { if (druidMetadataStorageTablesConfig != null) { return druidMetadataStorageTablesConfig; } - final String base = HiveConf - .getVar(getConf(), HiveConf.ConfVars.DRUID_METADATA_BASE); + final String base = HiveConf.getVar(getConf(), HiveConf.ConfVars.DRUID_METADATA_BASE); druidMetadataStorageTablesConfig = MetadataStorageTablesConfig.fromBase(base); return druidMetadataStorageTablesConfig; } @@ -954,7 +818,8 @@ private SQLMetadataConnector buildConnector() { final String password = HiveConf.getVar(getConf(), HiveConf.ConfVars.DRUID_METADATA_DB_PASSWORD); final String uri = HiveConf.getVar(getConf(), HiveConf.ConfVars.DRUID_METADATA_DB_URI); LOG.debug("Supplying SQL Connector with DB type {}, URI {}, User {}", dbType, uri, username); - final Supplier storageConnectorConfigSupplier = + @SuppressWarnings("Guava") final Supplier + storageConnectorConfigSupplier = Suppliers.ofInstance(new MetadataStorageConnectorConfig() { @Override public String getConnectURI() { return uri; @@ -968,27 +833,32 @@ private SQLMetadataConnector buildConnector() { return Strings.emptyToNull(password); } }); - if (dbType.equals("mysql")) { - connector = new MySQLConnector(storageConnectorConfigSupplier, - Suppliers.ofInstance(getDruidMetadataStorageTablesConfig()), new MySQLConnectorConfig() - ); - } else if (dbType.equals("postgresql")) { - connector = new PostgreSQLConnector(storageConnectorConfigSupplier, - Suppliers.ofInstance(getDruidMetadataStorageTablesConfig()) - ); - - } else if (dbType.equals("derby")) { - connector = new DerbyConnector(new DerbyMetadataStorage(storageConnectorConfigSupplier.get()), - storageConnectorConfigSupplier, Suppliers.ofInstance(getDruidMetadataStorageTablesConfig()) - ); - } else { + switch (dbType) { + case "mysql": + connector = + new MySQLConnector(storageConnectorConfigSupplier, + Suppliers.ofInstance(getDruidMetadataStorageTablesConfig()), + new MySQLConnectorConfig()); + break; + case "postgresql": + connector = + new PostgreSQLConnector(storageConnectorConfigSupplier, + Suppliers.ofInstance(getDruidMetadataStorageTablesConfig())); + + break; + case "derby": + connector = + new DerbyConnector(new DerbyMetadataStorage(storageConnectorConfigSupplier.get()), + storageConnectorConfigSupplier, + Suppliers.ofInstance(getDruidMetadataStorageTablesConfig())); + break; + default: throw new IllegalStateException(String.format("Unknown metadata storage type [%s]", dbType)); } return connector; } - @VisibleForTesting - protected String makeStagingName() { + @VisibleForTesting String makeStagingName() { return ".staging-".concat(getUniqueId().replace(":", "")); } @@ -1018,23 +888,22 @@ private String getRootWorkingDir() { } private static HttpClient makeHttpClient(Lifecycle lifecycle) { - final int numConnection = HiveConf - .getIntVar(SessionState.getSessionConf(), - HiveConf.ConfVars.HIVE_DRUID_NUM_HTTP_CONNECTION - ); - final Period readTimeout = new Period( - HiveConf.getVar(SessionState.getSessionConf(), - HiveConf.ConfVars.HIVE_DRUID_HTTP_READ_TIMEOUT - )); + final int + numConnection = + HiveConf.getIntVar(SessionState.getSessionConf(), HiveConf.ConfVars.HIVE_DRUID_NUM_HTTP_CONNECTION); + final Period + readTimeout = + new Period(HiveConf.getVar(SessionState.getSessionConf(), HiveConf.ConfVars.HIVE_DRUID_HTTP_READ_TIMEOUT)); LOG.info("Creating Druid HTTP client with {} max parallel connections and {}ms read timeout", - numConnection, readTimeout.toStandardDuration().getMillis() - ); - - final HttpClient httpClient = HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(numConnection) - .withReadTimeout(new Period(readTimeout).toStandardDuration()).build(), - lifecycle - ); + numConnection, + readTimeout.toStandardDuration().getMillis()); + + final HttpClient + httpClient = + HttpClientInit.createClient(HttpClientConfig.builder() + .withNumConnections(numConnection) + .withReadTimeout(new Period(readTimeout).toStandardDuration()) + .build(), lifecycle); if (UserGroupInformation.isSecurityEnabled()) { LOG.info("building Kerberos Http Client"); return new KerberosHttpClient(httpClient); @@ -1046,96 +915,34 @@ public static HttpClient getHttpClient() { return HTTP_CLIENT; } - @Override - public void preAlterTable(Table table, EnvironmentContext context) throws MetaException { - String alterOpType = - context == null ? null : context.getProperties().get(ALTER_TABLE_OPERATION_TYPE); + @Override public void preAlterTable(Table table, EnvironmentContext context) throws MetaException { + String alterOpType = context == null ? null : context.getProperties().get(ALTER_TABLE_OPERATION_TYPE); // alterOpType is null in case of stats update if (alterOpType != null && !allowedAlterTypes.contains(alterOpType)) { - throw new MetaException( - "ALTER TABLE can not be used for " + alterOpType + " to a non-native table "); + throw new MetaException("ALTER TABLE can not be used for " + alterOpType + " to a non-native table "); } - if(isKafkaStreamingTable(table)){ + if (DruidKafkaUtils.isKafkaStreamingTable(table)) { updateKafkaIngestion(table); } } - private static Boolean getBooleanProperty(Table table, String propertyName) { - String val = getTableProperty(table, propertyName); - if (val == null) { - return null; - } - return Boolean.parseBoolean(val); - } - - private static Integer getIntegerProperty(Table table, String propertyName) { - String val = getTableProperty(table, propertyName); - if (val == null) { - return null; - } - try { - return Integer.parseInt(val); - } catch (NumberFormatException e) { - throw new NumberFormatException(String - .format("Exception while parsing property[%s] with Value [%s] as Integer", propertyName, - val)); - } - } - - private static Long getLongProperty(Table table, String propertyName) { - String val = getTableProperty(table, propertyName); - if (val == null) { - return null; - } - try { - return Long.parseLong(val); - } catch (NumberFormatException e) { - throw new NumberFormatException(String - .format("Exception while parsing property[%s] with Value [%s] as Long", propertyName, - val)); - } - } - - private static Period getPeriodProperty(Table table, String propertyName) { - String val = getTableProperty(table, propertyName); - if (val == null) { - return null; - } - try { - return Period.parse(val); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException(String - .format("Exception while parsing property[%s] with Value [%s] as Period", propertyName, - val)); - } - } - - private static String getTableProperty(Table table, String propertyName) { - return table.getParameters().get(propertyName); - } - - private static boolean isKafkaStreamingTable(Table table){ - // For kafka Streaming tables it is mandatory to set a kafka topic. - return getTableProperty(table, Constants.KAFKA_TOPIC) != null; - } - private int getMaxRetryCount() { return HiveConf.getIntVar(getConf(), HiveConf.ConfVars.HIVE_DRUID_MAX_TRIES); } - @Override - public StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaException { - if(isKafkaStreamingTable(table)){ - KafkaSupervisorReport kafkaSupervisorReport = fetchKafkaSupervisorReport(table); - if(kafkaSupervisorReport == null){ - return DruidStorageHandlerInfo.UNREACHABLE; - } - return new DruidStorageHandlerInfo(kafkaSupervisorReport); - } - else - // TODO: Currently we do not expose any runtime info for non-streaming tables. - // In future extend this add more information regarding table status. - // e.g. Total size of segments in druid, loadstatus of table on historical nodes etc. + @Override public StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaException { + if (DruidKafkaUtils.isKafkaStreamingTable(table)) { + KafkaSupervisorReport kafkaSupervisorReport = fetchKafkaSupervisorReport(table); + if (kafkaSupervisorReport == null) { + return DruidStorageHandlerInfo.UNREACHABLE; + } + return new DruidStorageHandlerInfo(kafkaSupervisorReport); + } else + // TODO: Currently we do not expose any runtime info for non-streaming tables. + // In future extend this add more information regarding table status. + // e.g. Total size of segments in druid, load status of table on historical nodes etc. + { return null; + } } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java index f0e175057f..041cd34307 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerInfo.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hive.druid; -import io.druid.java.util.common.StringUtils; import org.apache.hadoop.hive.druid.json.KafkaSupervisorReport; import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo; @@ -25,15 +24,10 @@ /** * DruidStorageHandlerInfo provides a runtime information for DruidStorageHandler. */ -@SuppressWarnings("serial") public class DruidStorageHandlerInfo implements StorageHandlerInfo { - public static final StorageHandlerInfo UNREACHABLE = new StorageHandlerInfo() { - @Override - public String formatAsText() { - return "Druid Overlord is Unreachable, Runtime Status : unknown"; - } - }; + static final StorageHandlerInfo UNREACHABLE = + (StorageHandlerInfo) () -> "Druid Overlord is Unreachable, Runtime Status : unknown"; private final KafkaSupervisorReport kafkaSupervisorReport; @@ -44,28 +38,29 @@ public String formatAsText() { @Override public String formatAsText() { StringBuilder sb = new StringBuilder(); - sb.append("Druid Storage Handler Runtime Status for " + kafkaSupervisorReport.getId()); - sb.append("\n"); - sb.append("kafkaPartitions=" + kafkaSupervisorReport.getPayload().getPartitions()); - sb.append("\n"); - sb.append("activeTasks=" + kafkaSupervisorReport.getPayload().getActiveTasks()); - sb.append("\n"); - sb.append("publishingTasks=" + kafkaSupervisorReport.getPayload().getPublishingTasks()); + sb.append("Druid Storage Handler Runtime Status for ") + .append(kafkaSupervisorReport.getId()) + .append("\n") + .append("kafkaPartitions=") + .append(kafkaSupervisorReport.getPayload().getPartitions()) + .append("\n") + .append("activeTasks=") + .append(kafkaSupervisorReport.getPayload().getActiveTasks()) + .append("\n") + .append("publishingTasks=") + .append(kafkaSupervisorReport.getPayload().getPublishingTasks()); + if (kafkaSupervisorReport.getPayload().getLatestOffsets() != null) { - sb.append("\n"); - sb.append("latestOffsets=" + kafkaSupervisorReport.getPayload().getLatestOffsets()); + sb.append("\n").append("latestOffsets=").append(kafkaSupervisorReport.getPayload().getLatestOffsets()); } if (kafkaSupervisorReport.getPayload().getMinimumLag() != null) { - sb.append("\n"); - sb.append("minimumLag=" + kafkaSupervisorReport.getPayload().getMinimumLag()); + sb.append("\n").append("minimumLag=").append(kafkaSupervisorReport.getPayload().getMinimumLag()); } if (kafkaSupervisorReport.getPayload().getAggregateLag() != null) { - sb.append("\n"); - sb.append("aggregateLag=" + kafkaSupervisorReport.getPayload().getAggregateLag()); + sb.append("\n").append("aggregateLag=").append(kafkaSupervisorReport.getPayload().getAggregateLag()); } if (kafkaSupervisorReport.getPayload().getOffsetsLastUpdated() != null) { - sb.append("\n"); - sb.append("lastUpdateTime=" + kafkaSupervisorReport.getPayload().getOffsetsLastUpdated()); + sb.append("\n").append("lastUpdateTime=").append(kafkaSupervisorReport.getPayload().getOffsetsLastUpdated()); } return sb.toString(); } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java index c3e7e5df8d..daf786e055 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java @@ -30,7 +30,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import com.google.common.io.CharStreams; import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.StringDimensionSchema; import io.druid.jackson.DefaultObjectMapper; @@ -69,6 +68,7 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; +import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.indexing.granularity.GranularitySpec; @@ -92,6 +92,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; @@ -105,6 +106,7 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; import org.joda.time.Interval; +import org.joda.time.Period; import org.joda.time.chrono.ISOChronology; import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; @@ -119,35 +121,47 @@ import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.io.OutputStream; -import java.io.Reader; import java.net.InetAddress; import java.net.MalformedURLException; import java.net.URL; import java.net.UnknownHostException; import java.sql.SQLException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; import java.util.TimeZone; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; - +import java.util.stream.Collectors; /** * Utils class for Druid storage handler. */ public final class DruidStorageHandlerUtils { + private static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandlerUtils.class); + private static final String DRUID_ROLLUP = "druid.rollup"; + private static final String DRUID_QUERY_GRANULARITY = "druid.query.granularity"; + public static final String DRUID_QUERY_FETCH = "druid.query.fetch"; + static final String DRUID_SEGMENT_DIRECTORY = "druid.storage.storageDirectory"; + public static final String DRUID_SEGMENT_INTERMEDIATE_DIRECTORY = "druid.storage.storageDirectory.intermediate"; + public static final String DRUID_SEGMENT_VERSION = "druid.segment.version"; + public static final String DRUID_JOB_WORKING_DIRECTORY = "druid.job.workingDirectory"; + static final String KAFKA_TOPIC = "kafka.topic"; + static final String KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; + static final String DRUID_KAFKA_INGESTION_PROPERTY_PREFIX = "druid.kafka.ingestion."; + static final String DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX = DRUID_KAFKA_INGESTION_PROPERTY_PREFIX + "consumer."; + /* Kafka Ingestion state - valid values - START/STOP/RESET */ + static final String DRUID_KAFKA_INGESTION = "druid.kafka.ingestion"; private static final int NUM_RETRIES = 8; private static final int SECONDS_BETWEEN_RETRIES = 2; private static final int DEFAULT_FS_BUFFER_SIZE = 1 << 18; // 256KB @@ -157,12 +171,12 @@ public static final String DEFAULT_TIMESTAMP_COLUMN = "__time"; //Druid Json timestamp column name public static final String EVENT_TIMESTAMP_COLUMN = "timestamp"; - public static final String INDEX_ZIP = "index.zip"; - public static final String DESCRIPTOR_JSON = "descriptor.json"; - public static final Interval DEFAULT_INTERVAL = new Interval( - new DateTime("1900-01-01", ISOChronology.getInstanceUTC()), - new DateTime("3000-01-01", ISOChronology.getInstanceUTC()) - ).withChronology(ISOChronology.getInstanceUTC()); + static final String INDEX_ZIP = "index.zip"; + private static final String DESCRIPTOR_JSON = "descriptor.json"; + private static final Interval + DEFAULT_INTERVAL = + new Interval(new DateTime("1900-01-01", ISOChronology.getInstanceUTC()), + new DateTime("3000-01-01", ISOChronology.getInstanceUTC())).withChronology(ISOChronology.getInstanceUTC()); /** * Mapper to use to serialize/deserialize Druid objects (JSON) @@ -177,24 +191,24 @@ static { // This is needed for serde of PagingSpec as it uses JacksonInject for injecting SelectQueryConfig - InjectableValues.Std injectableValues = new InjectableValues.Std() - .addValue(SelectQueryConfig.class, new SelectQueryConfig(false)) - // Expressions macro table used when we deserialize the query from calcite plan - .addValue(ExprMacroTable.class, new ExprMacroTable(ImmutableList - .of(new LikeExprMacro(), - new RegexpExtractExprMacro(), - new TimestampCeilExprMacro(), - new TimestampExtractExprMacro(), - new TimestampFormatExprMacro(), - new TimestampParseExprMacro(), - new TimestampShiftExprMacro(), - new TimestampFloorExprMacro(), - new TrimExprMacro.BothTrimExprMacro(), - new TrimExprMacro.LeftTrimExprMacro(), - new TrimExprMacro.RightTrimExprMacro() - ))) - .addValue(ObjectMapper.class, JSON_MAPPER) - .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); + InjectableValues.Std + injectableValues = + new InjectableValues.Std().addValue(SelectQueryConfig.class, new SelectQueryConfig(false)) + // Expressions macro table used when we deserialize the query from calcite plan + .addValue(ExprMacroTable.class, + new ExprMacroTable(ImmutableList.of(new LikeExprMacro(), + new RegexpExtractExprMacro(), + new TimestampCeilExprMacro(), + new TimestampExtractExprMacro(), + new TimestampFormatExprMacro(), + new TimestampParseExprMacro(), + new TimestampShiftExprMacro(), + new TimestampFloorExprMacro(), + new TrimExprMacro.BothTrimExprMacro(), + new TrimExprMacro.LeftTrimExprMacro(), + new TrimExprMacro.RightTrimExprMacro()))) + .addValue(ObjectMapper.class, JSON_MAPPER) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); JSON_MAPPER.setInjectableValues(injectableValues); SMILE_MAPPER.setInjectableValues(injectableValues); @@ -206,10 +220,9 @@ JSON_MAPPER.setTimeZone(TimeZone.getTimeZone("UTC")); try { // No operation emitter will be used by some internal druid classes. - EmittingLogger.registerEmitter( - new ServiceEmitter("druid-hive-indexer", InetAddress.getLocalHost().getHostName(), - new NoopEmitter() - )); + EmittingLogger.registerEmitter(new ServiceEmitter("druid-hive-indexer", + InetAddress.getLocalHost().getHostName(), + new NoopEmitter())); } catch (UnknownHostException e) { throw Throwables.propagate(e); } @@ -218,98 +231,85 @@ /** * Used by druid to perform IO on indexes */ - public static final IndexIO INDEX_IO = + public static final IndexIO + INDEX_IO = new IndexIO(JSON_MAPPER, TmpFileSegmentWriteOutMediumFactory.instance(), () -> 0); /** * Used by druid to merge indexes */ - public static final IndexMergerV9 INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, - DruidStorageHandlerUtils.INDEX_IO,TmpFileSegmentWriteOutMediumFactory.instance() - ); + public static final IndexMergerV9 + INDEX_MERGER_V9 = + new IndexMergerV9(JSON_MAPPER, DruidStorageHandlerUtils.INDEX_IO, TmpFileSegmentWriteOutMediumFactory.instance()); /** * Generic Interner implementation used to read segments object from metadata storage */ - public static final Interner DATA_SEGMENT_INTERNER = Interners.newWeakInterner(); + private static final Interner DATA_SEGMENT_INTERNER = Interners.newWeakInterner(); /** * Method that creates a request for Druid query using SMILE format. * - * @param address - * @param query + * @param address of the host target. + * @param query druid query. * - * @return - * - * @throws IOException + * @return Request object to be submitted. */ - public static Request createSmileRequest(String address, io.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)) - .setHeader(HttpHeaders.Names.CONTENT_TYPE, SMILE_CONTENT_TYPE); + public static Request createSmileRequest(String address, io.druid.query.Query query) { + try { + return new Request(HttpMethod.POST, new URL(String.format("%s/druid/v2/", "http://" + address))).setContent( + SMILE_MAPPER.writeValueAsBytes(query)).setHeader(HttpHeaders.Names.CONTENT_TYPE, SMILE_CONTENT_TYPE); + } catch (MalformedURLException e) { + LOG.error("URL Malformed address {}", address); + throw new RuntimeException(e); + } catch (JsonProcessingException e) { + LOG.error("can not Serialize the Query [{}]", query.toString()); + throw new RuntimeException(e); + } } /** * Method that submits a request to an Http address and retrieves the result. * The caller is responsible for closing the stream once it finishes consuming it. * - * @param client - * @param request + * @param client Http Client will be used to submit request. + * @param request Http request to be submitted. * - * @return + * @return response object. * - * @throws IOException + * @throws IOException in case of request IO error. */ - public static InputStream submitRequest(HttpClient client, Request request) - throws IOException { - InputStream response; + public static InputStream submitRequest(HttpClient client, Request request) throws IOException { try { - response = client.go(request, new InputStreamResponseHandler()).get(); - } catch (ExecutionException e) { - throw new IOException(e.getCause()); - } catch (InterruptedException e) { + return client.go(request, new InputStreamResponseHandler()).get(); + } catch (ExecutionException | InterruptedException e) { throw new IOException(e.getCause()); } - return response; - } - public static String getURL(HttpClient client, URL url) throws IOException { - try (Reader reader = new InputStreamReader( - DruidStorageHandlerUtils.submitRequest(client, new Request(HttpMethod.GET, url)))) { - return CharStreams.toString(reader); - } } - public static FullResponseHolder getResponseFromCurrentLeader(HttpClient client, Request request, - FullResponseHandler fullResponseHandler) - throws ExecutionException, InterruptedException { - FullResponseHolder responseHolder = client.go(request, - fullResponseHandler).get(); + static FullResponseHolder getResponseFromCurrentLeader(HttpClient client, + Request request, + FullResponseHandler fullResponseHandler) throws ExecutionException, InterruptedException { + FullResponseHolder responseHolder = client.go(request, fullResponseHandler).get(); if (HttpResponseStatus.TEMPORARY_REDIRECT.equals(responseHolder.getStatus())) { String redirectUrlStr = responseHolder.getResponse().headers().get("Location"); - LOG.debug("Request[%s] received redirect response to location [%s].", request.getUrl(), - redirectUrlStr); + LOG.debug("Request[%s] received redirect response to location [%s].", request.getUrl(), redirectUrlStr); final URL redirectUrl; try { redirectUrl = new URL(redirectUrlStr); } catch (MalformedURLException ex) { - throw new ExecutionException( - String.format( - "Malformed redirect location is found in response from url[%s], new location[%s].", - request.getUrl(), - redirectUrlStr), - ex - ); + throw new ExecutionException(String.format( + "Malformed redirect location is found in response from url[%s], new location[%s].", + request.getUrl(), + redirectUrlStr), ex); } - responseHolder = client.go(withUrl(request, redirectUrl), - fullResponseHandler).get(); + responseHolder = client.go(withUrl(request, redirectUrl), fullResponseHandler).get(); } return responseHolder; } - private static Request withUrl(Request old, URL url) - { + private static Request withUrl(Request old, URL url) { Request req = new Request(old.getMethod(), url); req.addHeaderValues(old.getHeaders()); if (old.hasContent()) { @@ -327,61 +327,41 @@ private static Request withUrl(Request old, URL url) * * @throws IOException can be for the case we did not produce data. */ - public static List getCreatedSegments(Path taskDir, Configuration conf) - throws IOException { + public static List getCreatedSegments(Path taskDir, Configuration conf) throws IOException { ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); FileSystem fs = taskDir.getFileSystem(conf); FileStatus[] fss; fss = fs.listStatus(taskDir); for (FileStatus fileStatus : fss) { - final DataSegment segment = JSON_MAPPER - .readValue((InputStream) fs.open(fileStatus.getPath()), DataSegment.class); + final DataSegment segment = JSON_MAPPER.readValue((InputStream) fs.open(fileStatus.getPath()), DataSegment.class); publishedSegmentsBuilder.add(segment); } return publishedSegmentsBuilder.build(); } /** - * This function will write to filesystem serialized from of segment descriptor - * if an existing file exists it will try to replace it. + * Writes to filesystem serialized form of segment descriptor if an existing file exists it will try to replace it. * - * @param outputFS filesystem - * @param segment DataSegment object - * @param descriptorPath path + * @param outputFS filesystem. + * @param segment DataSegment object. + * @param descriptorPath path. * - * @throws IOException + * @throws IOException in case any IO issues occur. */ - public static void writeSegmentDescriptor( - final FileSystem outputFS, - final DataSegment segment, - final Path descriptorPath - ) - throws IOException { - final DataPusher descriptorPusher = (DataPusher) RetryProxy.create( - DataPusher.class, () -> { - try { - if (outputFS.exists(descriptorPath)) { - if (!outputFS.delete(descriptorPath, false)) { - throw new IOException( - String.format("Failed to delete descriptor at [%s]", descriptorPath)); - } - } - try (final OutputStream descriptorOut = outputFS.create( - descriptorPath, - true, - DEFAULT_FS_BUFFER_SIZE - )) { - JSON_MAPPER.writeValue(descriptorOut, segment); - descriptorOut.flush(); - } - } catch (RuntimeException | IOException ex) { - throw ex; - } - return -1; - }, - RetryPolicies - .exponentialBackoffRetry(NUM_RETRIES, SECONDS_BETWEEN_RETRIES, TimeUnit.SECONDS) - ); + public static void writeSegmentDescriptor(final FileSystem outputFS, + final DataSegment segment, + final Path descriptorPath) throws IOException { + final DataPusher descriptorPusher = (DataPusher) RetryProxy.create(DataPusher.class, () -> { + if (outputFS.exists(descriptorPath)) { + if (!outputFS.delete(descriptorPath, false)) { + throw new IOException(String.format("Failed to delete descriptor at [%s]", descriptorPath)); + } + } + try (final OutputStream descriptorOut = outputFS.create(descriptorPath, true, DEFAULT_FS_BUFFER_SIZE)) { + JSON_MAPPER.writeValue(descriptorOut, segment); + descriptorOut.flush(); + } + }, RetryPolicies.exponentialBackoffRetry(NUM_RETRIES, SECONDS_BETWEEN_RETRIES, TimeUnit.SECONDS)); descriptorPusher.push(); } @@ -391,23 +371,17 @@ public static void writeSegmentDescriptor( * * @return all the active data sources in the metadata storage */ - public static Collection getAllDataSourceNames(SQLMetadataConnector connector, - final MetadataStorageTablesConfig metadataStorageTablesConfig - ) { - return connector.getDBI().withHandle( - (HandleCallback>) handle -> handle.createQuery( - String.format("SELECT DISTINCT(datasource) FROM %s WHERE used = true", - metadataStorageTablesConfig.getSegmentsTable() - )) - .fold(Lists.newArrayList(), - (druidDataSources, stringObjectMap, foldController, statementContext) -> { - druidDataSources.add( - MapUtils.getString(stringObjectMap, "datasource") - ); - return druidDataSources; - } - ) - ); + static Collection getAllDataSourceNames(SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig) { + return connector.getDBI() + .withHandle((HandleCallback>) handle -> handle.createQuery(String.format( + "SELECT DISTINCT(datasource) FROM %s WHERE used = true", + metadataStorageTablesConfig.getSegmentsTable())) + .fold(Lists.newArrayList(), + (druidDataSources, stringObjectMap, foldController, statementContext) -> { + druidDataSources.add(MapUtils.getString(stringObjectMap, "datasource")); + return druidDataSources; + })); } /** @@ -417,21 +391,19 @@ public static void writeSegmentDescriptor( * * @return true if the data source was successfully disabled false otherwise */ - public static boolean disableDataSource(SQLMetadataConnector connector, - final MetadataStorageTablesConfig metadataStorageTablesConfig, final String dataSource - ) { + static boolean disableDataSource(SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig, + final String dataSource) { try { if (!getAllDataSourceNames(connector, metadataStorageTablesConfig).contains(dataSource)) { LOG.warn("Cannot delete data source {}, does not exist", dataSource); return false; } - connector.getDBI().withHandle( - (HandleCallback) handle -> { - disableDataSourceWithHandle(handle, metadataStorageTablesConfig, dataSource); - return null; - } - ); + connector.getDBI().withHandle((HandleCallback) handle -> { + disableDataSourceWithHandle(handle, metadataStorageTablesConfig, dataSource); + return null; + }); } catch (Exception e) { LOG.error(String.format("Error removing dataSource %s", dataSource), e); @@ -456,153 +428,128 @@ public static boolean disableDataSource(SQLMetadataConnector connector, * @return List of successfully published Druid segments. * This list has the updated versions and metadata about segments after move and timeline sorting * - * @throws CallbackFailedException + * @throws CallbackFailedException in case the connector can not add the segment to the DB. */ - public static List publishSegmentsAndCommit(final SQLMetadataConnector connector, - final MetadataStorageTablesConfig metadataStorageTablesConfig, - final String dataSource, - final List segments, - boolean overwrite, - Configuration conf, - DataSegmentPusher dataSegmentPusher - ) throws CallbackFailedException { - return connector.getDBI().inTransaction( - (handle, transactionStatus) -> { - // We create the timeline for the existing and new segments - VersionedIntervalTimeline timeline; - if (overwrite) { - // If we are overwriting, we disable existing sources - disableDataSourceWithHandle(handle, metadataStorageTablesConfig, dataSource); - - // When overwriting, we just start with empty timeline, - // as we are overwriting segments with new versions - timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - } else { - // Append Mode - if (segments.isEmpty()) { - // If there are no new segments, we can just bail out - return Collections.EMPTY_LIST; - } - // Otherwise, build a timeline of existing segments in metadata storage - Interval indexedInterval = JodaUtils - .umbrellaInterval(Iterables.transform(segments, - input -> input.getInterval() - )); - LOG.info("Building timeline for umbrella Interval [{}]", indexedInterval); - timeline = getTimelineForIntervalWithHandle( - handle, dataSource, indexedInterval, metadataStorageTablesConfig); - } - - final List finalSegmentsToPublish = Lists.newArrayList(); - for (DataSegment segment : segments) { - List> existingChunks = timeline - .lookup(segment.getInterval()); - if (existingChunks.size() > 1) { - // Not possible to expand since we have more than one chunk with a single segment. - // This is the case when user wants to append a segment with coarser granularity. - // e.g If metadata storage already has segments for with granularity HOUR and segments to append have DAY granularity. - // Druid shard specs does not support multiple partitions for same interval with different granularity. - throw new IllegalStateException( - String.format( - "Cannot allocate new segment for dataSource[%s], interval[%s], already have [%,d] chunks. Not possible to append new segment.", - dataSource, - segment.getInterval(), - existingChunks.size() - ) - ); - } - // Find out the segment with latest version and maximum partition number - SegmentIdentifier max = null; - final ShardSpec newShardSpec; - final String newVersion; - if (!existingChunks.isEmpty()) { - // Some existing chunk, Find max - TimelineObjectHolder existingHolder = Iterables - .getOnlyElement(existingChunks); - for (PartitionChunk existing : existingHolder.getObject()) { - if (max == null || - max.getShardSpec().getPartitionNum() < existing.getObject() - .getShardSpec() - .getPartitionNum()) { - max = SegmentIdentifier.fromDataSegment(existing.getObject()); - } - } - } - - if (max == null) { - // No existing shard present in the database, use the current version. - newShardSpec = segment.getShardSpec(); - newVersion = segment.getVersion(); - } else { - // use version of existing max segment to generate new shard spec - newShardSpec = getNextPartitionShardSpec(max.getShardSpec()); - newVersion = max.getVersion(); - } - DataSegment publishedSegment = publishSegmentWithShardSpec( - segment, - newShardSpec, - newVersion, - getPath(segment).getFileSystem(conf), - dataSegmentPusher - ); - finalSegmentsToPublish.add(publishedSegment); - timeline.add( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().createChunk(publishedSegment) - ); - - } - - // Publish new segments to metadata storage - final PreparedBatch batch = handle.prepareBatch( - String.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - metadataStorageTablesConfig.getSegmentsTable() - ) - - ); - - for (final DataSegment segment : finalSegmentsToPublish) { - - batch.add( - new ImmutableMap.Builder() - .put("id", segment.getIdentifier()) - .put("dataSource", segment.getDataSource()) - .put("created_date", new DateTime().toString()) - .put("start", segment.getInterval().getStart().toString()) - .put("end", segment.getInterval().getEnd().toString()) - .put("partitioned", - (segment.getShardSpec() instanceof NoneShardSpec) ? - false : - true - ) - .put("version", segment.getVersion()) - .put("used", true) - .put("payload", JSON_MAPPER.writeValueAsBytes(segment)) - .build() - ); - - LOG.info("Published {}", segment.getIdentifier()); - } - batch.execute(); - - return finalSegmentsToPublish; + @SuppressWarnings("unchecked") static List publishSegmentsAndCommit(final SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig, + final String dataSource, + final List segments, + boolean overwrite, + Configuration conf, + DataSegmentPusher dataSegmentPusher) throws CallbackFailedException { + return connector.getDBI().inTransaction((handle, transactionStatus) -> { + // We create the timeline for the existing and new segments + VersionedIntervalTimeline timeline; + if (overwrite) { + // If we are overwriting, we disable existing sources + disableDataSourceWithHandle(handle, metadataStorageTablesConfig, dataSource); + + // When overwriting, we just start with empty timeline, + // as we are overwriting segments with new versions + timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + } else { + // Append Mode + if (segments.isEmpty()) { + // If there are no new segments, we can just bail out + return Collections.EMPTY_LIST; + } + // Otherwise, build a timeline of existing segments in metadata storage + Interval + indexedInterval = + JodaUtils.umbrellaInterval(segments.stream().map(DataSegment::getInterval).collect(Collectors.toList())); + LOG.info("Building timeline for umbrella Interval [{}]", indexedInterval); + timeline = getTimelineForIntervalWithHandle(handle, dataSource, indexedInterval, metadataStorageTablesConfig); + } + + final List finalSegmentsToPublish = Lists.newArrayList(); + for (DataSegment segment : segments) { + List> existingChunks = timeline.lookup(segment.getInterval()); + if (existingChunks.size() > 1) { + // Not possible to expand since we have more than one chunk with a single segment. + // This is the case when user wants to append a segment with coarser granularity. + // e.g If metadata storage already has segments for with granularity HOUR and segments to append have DAY granularity. + // Druid shard specs does not support multiple partitions for same interval with different granularity. + throw new IllegalStateException(String.format( + "Cannot allocate new segment for dataSource[%s], interval[%s], already have [%,d] chunks. Not possible to append new segment.", + dataSource, + segment.getInterval(), + existingChunks.size())); + } + // Find out the segment with latest version and maximum partition number + SegmentIdentifier max = null; + final ShardSpec newShardSpec; + final String newVersion; + if (!existingChunks.isEmpty()) { + // Some existing chunk, Find max + TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); + for (PartitionChunk existing : existingHolder.getObject()) { + if (max == null || max.getShardSpec().getPartitionNum() < existing.getObject() + .getShardSpec() + .getPartitionNum()) { + max = SegmentIdentifier.fromDataSegment(existing.getObject()); } - ); + } + } + + if (max == null) { + // No existing shard present in the database, use the current version. + newShardSpec = segment.getShardSpec(); + newVersion = segment.getVersion(); + } else { + // use version of existing max segment to generate new shard spec + newShardSpec = getNextPartitionShardSpec(max.getShardSpec()); + newVersion = max.getVersion(); + } + DataSegment + publishedSegment = + publishSegmentWithShardSpec(segment, + newShardSpec, + newVersion, + getPath(segment).getFileSystem(conf), + dataSegmentPusher); + finalSegmentsToPublish.add(publishedSegment); + timeline.add(publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().createChunk(publishedSegment)); + + } + + // Publish new segments to metadata storage + final PreparedBatch + batch = + handle.prepareBatch(String.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + metadataStorageTablesConfig.getSegmentsTable()) + + ); + + for (final DataSegment segment : finalSegmentsToPublish) { + + batch.add(new ImmutableMap.Builder().put("id", segment.getIdentifier()) + .put("dataSource", segment.getDataSource()) + .put("created_date", new DateTime().toString()) + .put("start", segment.getInterval().getStart().toString()) + .put("end", segment.getInterval().getEnd().toString()) + .put("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) + .put("version", segment.getVersion()) + .put("used", true) + .put("payload", JSON_MAPPER.writeValueAsBytes(segment)) + .build()); + + LOG.info("Published {}", segment.getIdentifier()); + } + batch.execute(); + + return finalSegmentsToPublish; + }); } - public static void disableDataSourceWithHandle(Handle handle, - MetadataStorageTablesConfig metadataStorageTablesConfig, String dataSource - ) { - handle.createStatement( - String.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", - metadataStorageTablesConfig.getSegmentsTable() - ) - ) - .bind("dataSource", dataSource) - .execute(); + private static void disableDataSourceWithHandle(Handle handle, + MetadataStorageTablesConfig metadataStorageTablesConfig, + String dataSource) { + handle.createStatement(String.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", + metadataStorageTablesConfig.getSegmentsTable())).bind("dataSource", dataSource).execute(); } /** @@ -612,41 +559,31 @@ public static void disableDataSourceWithHandle(Handle handle, * * @return List of all data segments part of the given data source */ - public static List getDataSegmentList(final SQLMetadataConnector connector, - final MetadataStorageTablesConfig metadataStorageTablesConfig, final String dataSource - ) { - List segmentList = connector.retryTransaction( - (handle, status) -> handle - .createQuery(String.format( - "SELECT payload FROM %s WHERE dataSource = :dataSource", - metadataStorageTablesConfig.getSegmentsTable() - )) - .setFetchSize(getStreamingFetchSize(connector)) - .bind("dataSource", dataSource) - .map(ByteArrayMapper.FIRST) - .fold( - new ArrayList<>(), - (Folder3, byte[]>) (accumulator, payload, control, ctx) -> { - try { - final DataSegment segment = DATA_SEGMENT_INTERNER.intern( - JSON_MAPPER.readValue( - payload, - DataSegment.class - )); - - accumulator.add(segment); - return accumulator; - } catch (Exception e) { - throw new SQLException(e.toString()); - } - } - ) - , 3, DEFAULT_MAX_TRIES); - return segmentList; + static List getDataSegmentList(final SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig, + final String dataSource) { + return connector.retryTransaction((handle, status) -> handle.createQuery(String.format( + "SELECT payload FROM %s WHERE dataSource = :dataSource", + metadataStorageTablesConfig.getSegmentsTable())) + .setFetchSize(getStreamingFetchSize(connector)) + .bind("dataSource", dataSource) + .map(ByteArrayMapper.FIRST) + .fold(new ArrayList<>(), (Folder3, byte[]>) (accumulator, payload, control, ctx) -> { + try { + final DataSegment + segment = + DATA_SEGMENT_INTERNER.intern(JSON_MAPPER.readValue(payload, DataSegment.class)); + + accumulator.add(segment); + return accumulator; + } catch (Exception e) { + throw new SQLException(e.toString()); + } + }), 3, DEFAULT_MAX_TRIES); } /** - * @param connector + * @param connector SQL DBI connector. * * @return streaming fetch size. */ @@ -658,112 +595,149 @@ private static int getStreamingFetchSize(SQLMetadataConnector connector) { } /** - * @param pushedSegment - * @param segmentsDescriptorDir + * @param pushedSegment the pushed data segment object + * @param segmentsDescriptorDir actual directory path for descriptors. * * @return a sanitize file name */ - public static Path makeSegmentDescriptorOutputPath(DataSegment pushedSegment, - Path segmentsDescriptorDir - ) { - return new Path( - segmentsDescriptorDir, - String.format("%s.json", pushedSegment.getIdentifier().replace(":", "")) - ); + public static Path makeSegmentDescriptorOutputPath(DataSegment pushedSegment, Path segmentsDescriptorDir) { + return new Path(segmentsDescriptorDir, String.format("%s.json", pushedSegment.getIdentifier().replace(":", ""))); } public static String createScanAllQuery(String dataSourceName, List columns) throws JsonProcessingException { final ScanQuery.ScanQueryBuilder scanQueryBuilder = ScanQuery.newScanQueryBuilder(); - final List intervals = Arrays.asList(DEFAULT_INTERVAL); - ScanQuery scanQuery = scanQueryBuilder - .dataSource(dataSourceName) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) - .intervals(new MultipleIntervalSegmentSpec(intervals)) - .columns(columns) - .build(); + final List intervals = Collections.singletonList(DEFAULT_INTERVAL); + ScanQuery + scanQuery = + scanQueryBuilder.dataSource(dataSourceName) + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .intervals(new MultipleIntervalSegmentSpec(intervals)) + .columns(columns) + .build(); return JSON_MAPPER.writeValueAsString(scanQuery); } + + static Boolean getBooleanProperty(Table table, String propertyName) { + String val = getTableProperty(table, propertyName); + if (val == null) { + return null; + } + return Boolean.parseBoolean(val); + } + + static Integer getIntegerProperty(Table table, String propertyName) { + String val = getTableProperty(table, propertyName); + if (val == null) { + return null; + } + try { + return Integer.parseInt(val); + } catch (NumberFormatException e) { + throw new NumberFormatException(String.format("Exception while parsing property[%s] with Value [%s] as Integer", + propertyName, + val)); + } + } + + static Long getLongProperty(Table table, String propertyName) { + String val = getTableProperty(table, propertyName); + if (val == null) { + return null; + } + try { + return Long.parseLong(val); + } catch (NumberFormatException e) { + throw new NumberFormatException(String.format("Exception while parsing property[%s] with Value [%s] as Long", + propertyName, + val)); + } + } + + static Period getPeriodProperty(Table table, String propertyName) { + String val = getTableProperty(table, propertyName); + if (val == null) { + return null; + } + try { + return Period.parse(val); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException(String.format("Exception while parsing property[%s] with Value [%s] as Period", + propertyName, + val)); + } + } + + static String getTableProperty(Table table, String propertyName) { + return table.getParameters().get(propertyName); + } + /** * Simple interface for retry operations */ public interface DataPusher { - long push() throws IOException; + void push() throws IOException; } // Thanks, HBase Storage handler - public static void addDependencyJars(Configuration conf, Class... classes) throws IOException { + @SuppressWarnings("SameParameterValue") static void addDependencyJars(Configuration conf, Class... classes) throws IOException { FileSystem localFs = FileSystem.getLocal(conf); - Set jars = new HashSet(); - jars.addAll(conf.getStringCollection("tmpjars")); + Set jars = new HashSet<>(conf.getStringCollection("tmpjars")); for (Class clazz : classes) { if (clazz == null) { continue; } - String path = Utilities.jarFinderGetJar(clazz); + final String path = Utilities.jarFinderGetJar(clazz); if (path == null) { - throw new RuntimeException( - "Could not find jar for class " + clazz + " in order to ship it to the cluster."); + throw new RuntimeException("Could not find jar for class " + clazz + " in order to ship it to the cluster."); } if (!localFs.exists(new Path(path))) { throw new RuntimeException("Could not validate jar file " + path + " for class " + clazz); } - jars.add(path.toString()); + jars.add(path); } if (jars.isEmpty()) { return; } + //noinspection ToArrayCallWithZeroLengthArrayArgument conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[jars.size()]))); } - private static VersionedIntervalTimeline getTimelineForIntervalWithHandle( - final Handle handle, - final String dataSource, - final Interval interval, - final MetadataStorageTablesConfig dbTables - ) throws IOException { - Query> sql = handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE used = true AND dataSource = ? AND start <= ? AND \"end\" >= ?", - dbTables.getSegmentsTable() - ) - ).bind(0, dataSource) + private static VersionedIntervalTimeline getTimelineForIntervalWithHandle(final Handle handle, + final String dataSource, + final Interval interval, + final MetadataStorageTablesConfig dbTables) throws IOException { + Query> + sql = + handle.createQuery(String.format( + "SELECT payload FROM %s WHERE used = true AND dataSource = ? AND start <= ? AND \"end\" >= ?", + dbTables.getSegmentsTable())) + .bind(0, dataSource) .bind(1, interval.getEnd().toString()) .bind(2, interval.getStart().toString()); - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - Ordering.natural() - ); - final ResultIterator dbSegments = sql - .map(ByteArrayMapper.FIRST) - .iterator(); - try { + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + try (ResultIterator dbSegments = sql.map(ByteArrayMapper.FIRST).iterator()) { while (dbSegments.hasNext()) { final byte[] payload = dbSegments.next(); - DataSegment segment = JSON_MAPPER.readValue( - payload, - DataSegment.class - ); - timeline.add(segment.getInterval(), segment.getVersion(), - segment.getShardSpec().createChunk(segment) - ); + DataSegment segment = JSON_MAPPER.readValue(payload, DataSegment.class); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); } - } finally { - dbSegments.close(); } return timeline; } - public static DataSegmentPusher createSegmentPusherForDirectory(String segmentDirectory, - Configuration configuration) throws IOException { + public static DataSegmentPusher createSegmentPusherForDirectory(String segmentDirectory, Configuration configuration) + throws IOException { final HdfsDataSegmentPusherConfig hdfsDataSegmentPusherConfig = new HdfsDataSegmentPusherConfig(); hdfsDataSegmentPusherConfig.setStorageDirectory(segmentDirectory); - return new HdfsDataSegmentPusher( - hdfsDataSegmentPusherConfig, configuration, JSON_MAPPER); + return new HdfsDataSegmentPusher(hdfsDataSegmentPusherConfig, configuration, JSON_MAPPER); } - public static DataSegment publishSegmentWithShardSpec(DataSegment segment, ShardSpec shardSpec, - String version, FileSystem fs, DataSegmentPusher dataSegmentPusher - ) throws IOException { + private static DataSegment publishSegmentWithShardSpec(DataSegment segment, + ShardSpec shardSpec, + String version, + FileSystem fs, + DataSegmentPusher dataSegmentPusher) throws IOException { boolean retry = true; DataSegment.Builder dataSegmentBuilder = new DataSegment.Builder(segment).version(version); Path finalPath = null; @@ -772,8 +746,9 @@ public static DataSegment publishSegmentWithShardSpec(DataSegment segment, Shard dataSegmentBuilder.shardSpec(shardSpec); final Path intermediatePath = getPath(segment); - finalPath = new Path(dataSegmentPusher.getPathForHadoop(), dataSegmentPusher - .makeIndexPathName(dataSegmentBuilder.build(), DruidStorageHandlerUtils.INDEX_ZIP)); + finalPath = + new Path(dataSegmentPusher.getPathForHadoop(), + dataSegmentPusher.makeIndexPathName(dataSegmentBuilder.build(), DruidStorageHandlerUtils.INDEX_ZIP)); // Create parent if it does not exist, recreation is not an error fs.mkdirs(finalPath.getParent()); @@ -784,16 +759,13 @@ public static DataSegment publishSegmentWithShardSpec(DataSegment segment, Shard retry = true; } else { throw new IOException(String.format( - "Failed to rename intermediate segment[%s] to final segment[%s] is not present.", - intermediatePath, - finalPath - )); + "Failed to rename intermediate segment[%s] to final segment[%s] is not present.", + intermediatePath, + finalPath)); } } } - DataSegment dataSegment = dataSegmentBuilder - .loadSpec(dataSegmentPusher.makeLoadSpec(finalPath.toUri())) - .build(); + DataSegment dataSegment = dataSegmentBuilder.loadSpec(dataSegmentPusher.makeLoadSpec(finalPath.toUri())).build(); writeSegmentDescriptor(fs, dataSegment, new Path(finalPath.getParent(), DruidStorageHandlerUtils.DESCRIPTOR_JSON)); @@ -804,61 +776,58 @@ private static ShardSpec getNextPartitionShardSpec(ShardSpec shardSpec) { if (shardSpec instanceof LinearShardSpec) { return new LinearShardSpec(shardSpec.getPartitionNum() + 1); } else if (shardSpec instanceof NumberedShardSpec) { - return new NumberedShardSpec(shardSpec.getPartitionNum(), - ((NumberedShardSpec) shardSpec).getPartitions() - ); + return new NumberedShardSpec(shardSpec.getPartitionNum(), ((NumberedShardSpec) shardSpec).getPartitions()); } else { // Druid only support appending more partitions to Linear and Numbered ShardSpecs. - throw new IllegalStateException( - String.format( - "Cannot expand shard spec [%s]", - shardSpec - ) - ); + throw new IllegalStateException(String.format("Cannot expand shard spec [%s]", shardSpec)); } } - public static Path getPath(DataSegment dataSegment) { - return new Path(String.valueOf(dataSegment.getLoadSpec().get("path"))); + static Path getPath(DataSegment dataSegment) { + return new Path(String.valueOf(Objects.requireNonNull(dataSegment.getLoadSpec()).get("path"))); } public static GranularitySpec getGranularitySpec(Configuration configuration, Properties tableProperties) { - final String segmentGranularity = + final String + segmentGranularity = tableProperties.getProperty(Constants.DRUID_SEGMENT_GRANULARITY) != null ? tableProperties.getProperty(Constants.DRUID_SEGMENT_GRANULARITY) : HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_DRUID_INDEXING_GRANULARITY); - final boolean rollup = tableProperties.getProperty(Constants.DRUID_ROLLUP) != null ? - Boolean.parseBoolean(tableProperties.getProperty(Constants.DRUID_SEGMENT_GRANULARITY)): - HiveConf.getBoolVar(configuration, HiveConf.ConfVars.HIVE_DRUID_ROLLUP); - return new UniformGranularitySpec( - Granularity.fromString(segmentGranularity), - Granularity.fromString( - tableProperties.getProperty(Constants.DRUID_QUERY_GRANULARITY) == null - ? "NONE" - : tableProperties.getProperty(Constants.DRUID_QUERY_GRANULARITY)), + final boolean + rollup = + tableProperties.getProperty(DRUID_ROLLUP) != null ? + Boolean.parseBoolean(tableProperties.getProperty(Constants.DRUID_SEGMENT_GRANULARITY)) : + HiveConf.getBoolVar(configuration, HiveConf.ConfVars.HIVE_DRUID_ROLLUP); + return new UniformGranularitySpec(Granularity.fromString(segmentGranularity), + Granularity.fromString(tableProperties.getProperty(DRUID_QUERY_GRANULARITY) == null ? + "NONE" : + tableProperties.getProperty(DRUID_QUERY_GRANULARITY)), rollup, - null - ); + null); } public static IndexSpec getIndexSpec(Configuration jc) { - IndexSpec indexSpec; + final BitmapSerdeFactory bitmapSerdeFactory; if ("concise".equals(HiveConf.getVar(jc, HiveConf.ConfVars.HIVE_DRUID_BITMAP_FACTORY_TYPE))) { - indexSpec = new IndexSpec(new ConciseBitmapSerdeFactory(), null, null, null); + bitmapSerdeFactory = new ConciseBitmapSerdeFactory(); } else { - indexSpec = new IndexSpec(new RoaringBitmapSerdeFactory(true), null, null, null); + bitmapSerdeFactory = new RoaringBitmapSerdeFactory(true); } - return indexSpec; + return new IndexSpec(bitmapSerdeFactory, + IndexSpec.DEFAULT_DIMENSION_COMPRESSION, + IndexSpec.DEFAULT_METRIC_COMPRESSION, + IndexSpec.DEFAULT_LONG_ENCODING); } - public static Pair, AggregatorFactory[]> getDimensionsAndAggregates(Configuration jc, List columnNames, + public static Pair, AggregatorFactory[]> getDimensionsAndAggregates(List columnNames, List columnTypes) { // Default, all columns that are not metrics or timestamp, are treated as dimensions final List dimensions = new ArrayList<>(); ImmutableList.Builder aggregatorFactoryBuilder = ImmutableList.builder(); for (int i = 0; i < columnTypes.size(); i++) { - final PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = ((PrimitiveTypeInfo) columnTypes - .get(i)).getPrimitiveCategory(); + final PrimitiveObjectInspector.PrimitiveCategory + primitiveCategory = + ((PrimitiveTypeInfo) columnTypes.get(i)).getPrimitiveCategory(); AggregatorFactory af; switch (primitiveCategory) { case BYTE: @@ -874,38 +843,39 @@ public static IndexSpec getIndexSpec(Configuration jc) { af = new DoubleSumAggregatorFactory(columnNames.get(i), columnNames.get(i)); break; case DECIMAL: - throw new UnsupportedOperationException( - String.format("Druid does not support decimal column type cast column " - + "[%s] to double", columnNames.get(i))); - + throw new UnsupportedOperationException(String.format("Druid does not support decimal column type cast column " + + "[%s] to double", columnNames.get(i))); case TIMESTAMP: // Granularity column String tColumnName = columnNames.get(i); - if (!tColumnName.equals(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME) && - !tColumnName.equals(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN)) { - throw new IllegalArgumentException( - "Dimension " + tColumnName + " does not have STRING type: " + - primitiveCategory); + if (!tColumnName.equals(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME) && !tColumnName.equals( + DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN)) { + throw new IllegalArgumentException("Dimension " + + tColumnName + + " does not have STRING type: " + + primitiveCategory); } continue; case TIMESTAMPLOCALTZ: // Druid timestamp column String tLocalTZColumnName = columnNames.get(i); if (!tLocalTZColumnName.equals(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN)) { - throw new IllegalArgumentException( - "Dimension " + tLocalTZColumnName + " does not have STRING type: " + - primitiveCategory); + throw new IllegalArgumentException("Dimension " + + tLocalTZColumnName + + " does not have STRING type: " + + primitiveCategory); } continue; default: // Dimension String dColumnName = columnNames.get(i); - if (PrimitiveObjectInspectorUtils.getPrimitiveGrouping(primitiveCategory) != - PrimitiveObjectInspectorUtils.PrimitiveGrouping.STRING_GROUP + if (PrimitiveObjectInspectorUtils.getPrimitiveGrouping(primitiveCategory) + != PrimitiveObjectInspectorUtils.PrimitiveGrouping.STRING_GROUP && primitiveCategory != PrimitiveObjectInspector.PrimitiveCategory.BOOLEAN) { - throw new IllegalArgumentException( - "Dimension " + dColumnName + " does not have STRING type: " + - primitiveCategory); + throw new IllegalArgumentException("Dimension " + + dColumnName + + " does not have STRING type: " + + primitiveCategory); } dimensions.add(new StringDimensionSchema(dColumnName)); continue; @@ -913,7 +883,6 @@ public static IndexSpec getIndexSpec(Configuration jc) { aggregatorFactoryBuilder.add(af); } ImmutableList aggregatorFactories = aggregatorFactoryBuilder.build(); - return Pair.of(dimensions, - aggregatorFactories.toArray(new AggregatorFactory[aggregatorFactories.size()])); + return Pair.of(dimensions, aggregatorFactories.toArray(new AggregatorFactory[0])); } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java index ecb4360623..8813ced299 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hive.druid.io; +import com.fasterxml.jackson.core.type.TypeReference; import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; @@ -64,9 +65,9 @@ import java.util.Map; import java.util.Properties; -public class DruidOutputFormat implements HiveOutputFormat { +public class DruidOutputFormat implements HiveOutputFormat { - protected static final Logger LOG = LoggerFactory.getLogger(DruidOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(DruidOutputFormat.class); @Override public FileSinkOperator.RecordWriter getHiveRecordWriter( @@ -89,7 +90,7 @@ final String dataSource = tableProperties.getProperty(Constants.DRUID_DATA_SOURCE) == null ? jc.get(Constants.DRUID_DATA_SOURCE) : tableProperties.getProperty(Constants.DRUID_DATA_SOURCE); - final String segmentDirectory = jc.get(Constants.DRUID_SEGMENT_INTERMEDIATE_DIRECTORY); + final String segmentDirectory = jc.get(DruidStorageHandlerUtils.DRUID_SEGMENT_INTERMEDIATE_DIRECTORY); final GranularitySpec granularitySpec = DruidStorageHandlerUtils.getGranularitySpec(jc, tableProperties); @@ -111,7 +112,7 @@ ArrayList columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty); Pair, AggregatorFactory[]> dimensionsAndAggregates = DruidStorageHandlerUtils - .getDimensionsAndAggregates(jc, columnNames, columnTypes); + .getDimensionsAndAggregates(columnNames, columnTypes); final InputRowParser inputRowParser = new MapInputRowParser(new TimeAndDimsParseSpec( new TimestampSpec(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, "auto", null), new DimensionsSpec(dimensionsAndAggregates.lhs, Lists @@ -121,8 +122,10 @@ ) )); - Map inputParser = DruidStorageHandlerUtils.JSON_MAPPER - .convertValue(inputRowParser, Map.class); + Map + inputParser = + DruidStorageHandlerUtils.JSON_MAPPER.convertValue(inputRowParser, new TypeReference>() { + }); final DataSchema dataSchema = new DataSchema( Preconditions.checkNotNull(dataSource, "Data source name is null"), @@ -133,8 +136,8 @@ DruidStorageHandlerUtils.JSON_MAPPER ); - final String workingPath = jc.get(Constants.DRUID_JOB_WORKING_DIRECTORY); - final String version = jc.get(Constants.DRUID_SEGMENT_VERSION); + final String workingPath = jc.get(DruidStorageHandlerUtils.DRUID_JOB_WORKING_DIRECTORY); + final String version = jc.get(DruidStorageHandlerUtils.DRUID_SEGMENT_VERSION); String basePersistDirectory = HiveConf .getVar(jc, HiveConf.ConfVars.HIVE_DRUID_BASE_PERSIST_DIRECTORY); if (Strings.isNullOrEmpty(basePersistDirectory)) { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java index 1c989c1a4d..0db06abd01 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java @@ -101,7 +101,7 @@ public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) @Override public List getSplits(JobContext context) throws IOException, InterruptedException { - return Arrays.asList(getInputSplits(context.getConfiguration())); + return Arrays.asList(getInputSplits(context.getConfiguration())); } @SuppressWarnings("deprecation") @@ -158,11 +158,11 @@ public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) case Query.SELECT: SelectQuery selectQuery = DruidStorageHandlerUtils.JSON_MAPPER.readValue( druidQuery, SelectQuery.class); - return distributeSelectQuery(conf, address, selectQuery, paths[0]); + return distributeSelectQuery(address, selectQuery, paths[0]); case Query.SCAN: ScanQuery scanQuery = DruidStorageHandlerUtils.JSON_MAPPER.readValue( druidQuery, ScanQuery.class); - return distributeScanQuery(conf, address, scanQuery, paths[0]); + return distributeScanQuery(address, scanQuery, paths[0]); default: throw new IOException("Druid query type not recognized"); } @@ -171,18 +171,17 @@ public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) /* New method that distributes the Select query by creating splits containing * information about different Druid nodes that have the data for the given * query. */ - private static HiveDruidSplit[] distributeSelectQuery(Configuration conf, String address, - SelectQuery query, Path dummyPath) throws IOException { + private static HiveDruidSplit[] distributeSelectQuery(String address, SelectQuery query, Path dummyPath) + throws IOException { // If it has a limit, we use it and we do not distribute the query - final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false); + final boolean isFetch = query.getContextBoolean(DruidStorageHandlerUtils.DRUID_QUERY_FETCH, false); if (isFetch) { - return new HiveDruidSplit[] { new HiveDruidSplit( - DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath, - new String[]{address} ) }; + return new HiveDruidSplit[] { new HiveDruidSplit(DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), + dummyPath, + new String[] { address }) }; } - final List segmentDescriptors = fetchLocatedSegmentDescriptors( - address, query); + final List segmentDescriptors = fetchLocatedSegmentDescriptors(address, query); // Create one input split for each segment final int numSplits = segmentDescriptors.size(); @@ -211,8 +210,8 @@ public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) /* New method that distributes the Scan query by creating splits containing * information about different Druid nodes that have the data for the given * query. */ - private static HiveDruidSplit[] distributeScanQuery(Configuration conf, String address, - ScanQuery query, Path dummyPath) throws IOException { + private static HiveDruidSplit[] distributeScanQuery(String address, ScanQuery query, Path dummyPath) + throws IOException { // If it has a limit, we use it and we do not distribute the query final boolean isFetch = query.getLimit() < Long.MAX_VALUE; if (isFetch) { @@ -288,7 +287,7 @@ private static String withQueryId(String druidQuery, String queryId) throws IOEx throws IOException { // We need to provide a different record reader for every type of Druid query. // The reason is that Druid results format is different for each type. - final DruidQueryRecordReader reader; + final DruidQueryRecordReader reader; final String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE); if (druidQueryType == null) { reader = new DruidScanQueryRecordReader(); // By default we use scan query as fallback. @@ -314,7 +313,7 @@ private static String withQueryId(String druidQuery, String queryId) throws IOEx if (druidQueryType == null) { return new DruidScanQueryRecordReader(); // By default, we use druid scan query as fallback. } - final DruidQueryRecordReader reader = + final DruidQueryRecordReader reader = getDruidQueryReader(druidQueryType); if (reader == null) { throw new IOException("Druid query type " + druidQueryType + " not recognized"); diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java index 400262a107..e835138256 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java @@ -17,13 +17,10 @@ */ package org.apache.hadoop.hive.druid.io; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.data.input.Committer; @@ -49,6 +46,7 @@ import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.hive.druid.serde.DruidWritable; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.RecordWriter; @@ -58,16 +56,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; +import java.util.stream.Collectors; public class DruidRecordWriter implements RecordWriter, - org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter { + FileSinkOperator.RecordWriter { protected static final Logger LOG = LoggerFactory.getLogger(DruidRecordWriter.class); private final DataSchema dataSchema; @@ -117,7 +117,7 @@ public DruidRecordWriter( this.fileSystem = Preconditions.checkNotNull(fileSystem, "file system is null"); this.segmentGranularity = this.dataSchema.getGranularitySpec() .getSegmentGranularity(); - committerSupplier = Suppliers.ofInstance(Committers.nil()); + committerSupplier = Suppliers.ofInstance(Committers.nil())::get; } /** @@ -200,18 +200,11 @@ private void pushSegments(List segmentsToPush) { ); } - final HashSet toPushSegmentsHashSet = new HashSet( - FluentIterable.from(segmentsToPush) - .transform(new Function() { - @Nullable - @Override - public String apply( - @Nullable SegmentIdentifier input - ) { - return input.getIdentifierAsString(); - } - }) - .toList()); + final Set + toPushSegmentsHashSet = + segmentsToPush.stream() + .map(SegmentIdentifier::getIdentifierAsString) + .collect(Collectors.toCollection(HashSet::new)); if (!pushedSegmentIdentifierHashSet.equals(toPushSegmentsHashSet)) { throw new IllegalStateException(String.format( @@ -243,7 +236,7 @@ public void write(Writable w) throws IOException { final long timestamp = (long) record.getValue().get(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN); final int partitionNumber = Math.toIntExact( - (long) record.getValue().getOrDefault(Constants.DRUID_SHARD_KEY_COL_NAME, -1l)); + (long) record.getValue().getOrDefault(Constants.DRUID_SHARD_KEY_COL_NAME, -1L)); final InputRow inputRow = new MapBasedInputRow(timestamp, dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(), record.getValue() @@ -272,19 +265,19 @@ public void write(Writable w) throws IOException { new LinearShardSpec(partitionNumber) ); } - } else if (currentOpenSegment == null) { + } else { currentOpenSegment = new SegmentIdentifier(dataSchema.getDataSource(), interval, tuningConfig.getVersioningPolicy().getVersion(interval), new LinearShardSpec(partitionNumber) ); } - appenderator.add(currentOpenSegment, inputRow, committerSupplier); + appenderator.add(currentOpenSegment, inputRow, committerSupplier::get); } else if (partitionNumber == -1 && maxPartitionSize != -1) { /*Case we are partitioning the segments based on time and max row per segment maxPartitionSize*/ appenderator - .add(getSegmentIdentifierAndMaybePush(timestamp), inputRow, committerSupplier); + .add(getSegmentIdentifierAndMaybePush(timestamp), inputRow, committerSupplier::get); } else { throw new IllegalArgumentException(String.format( "partitionNumber and maxPartitionSize should be mutually exclusive got partitionNum [%s] and maxPartitionSize [%s]", @@ -300,7 +293,7 @@ public void write(Writable w) throws IOException { @Override public void close(boolean abort) throws IOException { try { - if (abort == false) { + if (!abort) { final List segmentsToPush = Lists.newArrayList(); segmentsToPush.addAll(appenderator.getSegments()); pushSegments(segmentsToPush); diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java index 278301667d..f77480b1f3 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java @@ -36,7 +36,7 @@ // required for deserialization public HiveDruidSplit() { - super((Path) null, 0, 0, (String[]) null); + super(null, 0, 0, (String[]) null); } public HiveDruidSplit(String druidQuery, Path dummyPath, String hosts[]) { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java index 425a5bbfc5..6d9c1aaf13 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorIOConfig.java @@ -17,13 +17,11 @@ */ package org.apache.hadoop.hive.druid.json; -import io.druid.java.util.common.StringUtils; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; - +import io.druid.java.util.common.StringUtils; import org.joda.time.Duration; import org.joda.time.Period; @@ -46,8 +44,8 @@ private final Duration period; private final boolean useEarliestOffset; private final Duration completionTimeout; - private final Optional lateMessageRejectionPeriod; - private final Optional earlyMessageRejectionPeriod; + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private final Optional lateMessageRejectionPeriod; + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private final Optional earlyMessageRejectionPeriod; private final boolean skipOffsetGaps; @JsonCreator @@ -81,10 +79,10 @@ public KafkaSupervisorIOConfig( this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false; this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null - ? Optional.absent() + ? Optional.absent() : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null - ? Optional.absent() + ? Optional.absent() : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : false; } @@ -217,13 +215,13 @@ public boolean equals(Object o) { !completionTimeout.equals(that.completionTimeout) : that.completionTimeout != null) return false; - if (lateMessageRejectionPeriod != null ? + if (lateMessageRejectionPeriod.isPresent() ? !lateMessageRejectionPeriod.equals(that.lateMessageRejectionPeriod) : - that.lateMessageRejectionPeriod != null) + that.lateMessageRejectionPeriod.isPresent()) return false; - return earlyMessageRejectionPeriod != null ? + return earlyMessageRejectionPeriod.isPresent() ? earlyMessageRejectionPeriod.equals(that.earlyMessageRejectionPeriod) : - that.earlyMessageRejectionPeriod == null; + !that.earlyMessageRejectionPeriod.isPresent(); } @Override @@ -238,9 +236,9 @@ public int hashCode() { result = 31 * result + (useEarliestOffset ? 1 : 0); result = 31 * result + (completionTimeout != null ? completionTimeout.hashCode() : 0); result = 31 * result + - (lateMessageRejectionPeriod != null ? lateMessageRejectionPeriod.hashCode() : 0); + (lateMessageRejectionPeriod.isPresent() ? lateMessageRejectionPeriod.hashCode() : 0); result = 31 * result + - (earlyMessageRejectionPeriod != null ? earlyMessageRejectionPeriod.hashCode() : 0); + (earlyMessageRejectionPeriod.isPresent() ? earlyMessageRejectionPeriod.hashCode() : 0); result = 31 * result + (skipOffsetGaps ? 1 : 0); return result; } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java index 5a6756ecbf..eea453b024 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorReport.java @@ -17,9 +17,7 @@ */ 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; @@ -208,17 +206,6 @@ 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() { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java index a918df475d..47e6468fb1 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaSupervisorTuningConfig.java @@ -132,6 +132,7 @@ public Duration getOffsetFetchPeriod() @Override public String toString() { + //noinspection deprecation return "KafkaSupervisorTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java index 1ec8b5c238..05c98eb69b 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/KafkaTuningConfig.java @@ -47,7 +47,6 @@ private final int maxPendingPersists; private final IndexSpec indexSpec; private final boolean reportParseExceptions; - @Deprecated private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; @@ -60,7 +59,7 @@ public KafkaTuningConfig( @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, + @SuppressWarnings("unused") @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically @@ -88,22 +87,6 @@ public KafkaTuningConfig( : resetOffsetAutomatically; } - 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 - ); - } - @Override @JsonProperty public int getMaxRowsInMemory() @@ -166,7 +149,6 @@ public boolean isReportParseExceptions() return reportParseExceptions; } - @Deprecated @JsonProperty public long getHandoffConditionTimeout() { @@ -179,22 +161,6 @@ public boolean isResetOffsetAutomatically() return resetOffsetAutomatically; } - public KafkaTuningConfig withBasePersistDirectory(File dir) - { - return new KafkaTuningConfig( - maxRowsInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - dir, - maxPendingPersists, - indexSpec, - true, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically - ); - } - @Override public boolean equals(Object o) { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/security/DruidKerberosUtil.java druid-handler/src/java/org/apache/hadoop/hive/druid/security/DruidKerberosUtil.java index 84142afc08..cafeaa613f 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/security/DruidKerberosUtil.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/security/DruidKerberosUtil.java @@ -43,19 +43,18 @@ private static final Base64 base64codec = new Base64(0); - // A fair reentrant lock - private static ReentrantLock kerberosLock = new ReentrantLock(true); + private static final ReentrantLock kerberosLock = new ReentrantLock(true); /** * This method always needs to be called within a doAs block so that the client's TGT credentials * can be read from the Subject. * - * @return Kerberos Challenge String + * @return Kerberos Challenge String. * - * @throws Exception + * @throws AuthenticationException on authentication errors. */ - public static String kerberosChallenge(String server) throws AuthenticationException + static String kerberosChallenge(String server) throws AuthenticationException { kerberosLock.lock(); try { @@ -86,9 +85,7 @@ public static String kerberosChallenge(String server) throws AuthenticationExcep } } - - - public static HttpCookie getAuthCookie(CookieStore cookieStore, URI uri) + static HttpCookie getAuthCookie(CookieStore cookieStore, URI uri) { if (cookieStore == null) { return null; @@ -96,11 +93,10 @@ public static HttpCookie getAuthCookie(CookieStore cookieStore, URI uri) boolean isSSL = uri.getScheme().equals("https"); List cookies = cookieStore.getCookies(); - for (int i = 0; i < cookies.size(); i++) { + for (HttpCookie c : cookies) { // If this is a secured cookie and the current connection is non-secured, // then, skip this cookie. We need to skip this cookie because, the cookie // replay will not be transmitted to the server. - HttpCookie c = cookies.get(i); if (c.getSecure() && !isSSL) { continue; } @@ -111,7 +107,7 @@ public static HttpCookie getAuthCookie(CookieStore cookieStore, URI uri) return null; } - public static void removeAuthCookie(CookieStore cookieStore, URI uri) + static void removeAuthCookie(CookieStore cookieStore, URI uri) { HttpCookie authCookie = getAuthCookie(cookieStore, uri); if (authCookie != null) { @@ -120,7 +116,7 @@ public static void removeAuthCookie(CookieStore cookieStore, URI uri) } - public static boolean needToSendCredentials(CookieStore cookieStore, URI uri){ + static boolean needToSendCredentials(CookieStore cookieStore, URI uri){ return getAuthCookie(cookieStore, uri) == null; } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java index 4644df1fcc..7c78ffe6bb 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/security/KerberosHttpClient.java @@ -43,36 +43,28 @@ * 1/ Need to remove the authentication step since it not required * 2/ To avoid some un-needed transitive dependencies that can clash on the classpath like jetty-XX. */ -public class KerberosHttpClient extends AbstractHttpClient -{ +public class KerberosHttpClient extends AbstractHttpClient { protected static final org.slf4j.Logger log = LoggerFactory.getLogger(KerberosHttpClient.class); private final HttpClient delegate; private final CookieManager cookieManager; - public KerberosHttpClient(HttpClient delegate) - { + public KerberosHttpClient(HttpClient delegate) { this.delegate = delegate; this.cookieManager = new CookieManager(); } - @Override - public ListenableFuture go( - Request request, HttpResponseHandler httpResponseHandler, Duration duration - ) - { + @Override public ListenableFuture go(Request request, + HttpResponseHandler httpResponseHandler, + Duration duration) { final SettableFuture retVal = SettableFuture.create(); inner_go(request, httpResponseHandler, duration, retVal); return retVal; } - - private void inner_go( - final Request request, - final HttpResponseHandler httpResponseHandler, - final Duration duration, - final SettableFuture future - ) - { + private void inner_go(final Request request, + final HttpResponseHandler httpResponseHandler, + final Duration duration, + final SettableFuture future) { try { final String host = request.getUrl().getHost(); final URI uri = request.getUrl().toURI(); @@ -80,7 +72,7 @@ public KerberosHttpClient(HttpClient delegate) /* Cookies Manager is used to cache cookie returned by service. The goal us to avoid doing KDC requests for every request.*/ - Map> cookieMap = cookieManager.get(uri, Collections.>emptyMap()); + Map> cookieMap = cookieManager.get(uri, Collections.emptyMap()); for (Map.Entry> entry : cookieMap.entrySet()) { request.addHeaderValues(entry.getKey(), entry.getValue()); } @@ -88,41 +80,41 @@ public KerberosHttpClient(HttpClient delegate) if (DruidKerberosUtil.needToSendCredentials(cookieManager.getCookieStore(), uri)) { // No Cookies for requested URI, authenticate user and add authentication header - log.debug("No Auth Cookie found for URI{}. Existing Cookies{} Authenticating... ", uri, - cookieManager.getCookieStore().getCookies() - ); + log.debug("No Auth Cookie found for URI{}. Existing Cookies{} Authenticating... ", + uri, + cookieManager.getCookieStore().getCookies()); // Assuming that a valid UGI with kerberos cred is created by HS2 or LLAP UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); currentUser.checkTGTAndReloginFromKeytab(); log.debug("The user credential is {}", currentUser); - String challenge = currentUser.doAs(new PrivilegedExceptionAction() { - @Override public String run() throws Exception { - return DruidKerberosUtil.kerberosChallenge(host); - } - }); + String + challenge = + currentUser.doAs((PrivilegedExceptionAction) () -> DruidKerberosUtil.kerberosChallenge(host)); request.setHeader(HttpHeaders.Names.AUTHORIZATION, "Negotiate " + challenge); /* no reason to retry if the challenge ticket is not valid. */ should_retry_on_unauthorized_response = false; } else { /* In this branch we had already a cookie that did expire therefore we need to resend a valid Kerberos challenge*/ - log.debug("Found Auth Cookie found for URI {} cookie {}", uri, - DruidKerberosUtil.getAuthCookie(cookieManager.getCookieStore(), uri).toString() - ); + log.debug("Found Auth Cookie found for URI {} cookie {}", + uri, + DruidKerberosUtil.getAuthCookie(cookieManager.getCookieStore(), uri).toString()); should_retry_on_unauthorized_response = true; } - ListenableFuture> internalFuture = delegate.go(request, new RetryIfUnauthorizedResponseHandler( - new ResponseCookieHandler(request.getUrl().toURI(), cookieManager, httpResponseHandler - )), duration - ); + @SuppressWarnings("unchecked") ListenableFuture> + internalFuture = + delegate.go(request, + new RetryIfUnauthorizedResponseHandler(new ResponseCookieHandler(request.getUrl() + .toURI(), cookieManager, httpResponseHandler)), + duration); RetryResponseHolder responseHolder = internalFuture.get(); if (should_retry_on_unauthorized_response && responseHolder.shouldRetry()) { - log.debug("Preparing for Retry boolean {} and result {}, object{} ", - should_retry_on_unauthorized_response, responseHolder.shouldRetry(), responseHolder.getObj() - ); + log.debug("Preparing for Retry boolean {} and result {}, object{} ", true, + responseHolder.shouldRetry(), + responseHolder.getObj()); // remove Auth cookie DruidKerberosUtil.removeAuthCookie(cookieManager.getCookieStore(), uri); // clear existing cookie @@ -132,11 +124,9 @@ public KerberosHttpClient(HttpClient delegate) } else { future.set(responseHolder.getObj()); } - } - catch (Throwable e) { + } catch (Throwable e) { throw Throwables.propagate(e); } } - } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java index 221081ae7d..a316dc70de 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/security/ResponseCookieHandler.java @@ -39,7 +39,7 @@ private final CookieManager manager; private final HttpResponseHandler delegate; - public ResponseCookieHandler(URI uri, CookieManager manager, HttpResponseHandler delegate) + ResponseCookieHandler(URI uri, CookieManager manager, HttpResponseHandler delegate) { this.uri = uri; this.manager = manager; @@ -51,13 +51,12 @@ public ResponseCookieHandler(URI uri, CookieManager manager, HttpResponseHandler { try { final HttpHeaders headers = httpResponse.headers(); - manager.put(uri, Maps.asMap(headers.names(), input -> headers.getAll(input))); + manager.put(uri, Maps.asMap(headers.names(), headers::getAll)); + return delegate.handleResponse(httpResponse); } catch (IOException e) { log.error("Error while processing Cookies from header", e); - } - finally { - return delegate.handleResponse(httpResponse); + throw new RuntimeException(e); } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java index ff169e88c8..83cef164da 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryIfUnauthorizedResponseHandler.java @@ -46,8 +46,9 @@ public RetryIfUnauthorizedResponseHandler( log.debug("UnauthorizedResponseHandler - Got response status {}", httpResponse.getStatus()); if (httpResponse.getStatus().equals(HttpResponseStatus.UNAUTHORIZED)) { // Drain the buffer + //noinspection ResultOfMethodCallIgnored httpResponse.getContent().toString(); - return ClientResponse.unfinished(RetryResponseHolder.retry()); + return ClientResponse.unfinished(RetryResponseHolder.retry()); } else { return wrap(httpResponseHandler.handleResponse(httpResponse)); } @@ -59,6 +60,8 @@ public RetryIfUnauthorizedResponseHandler( ) { if (clientResponse.getObj().shouldRetry()) { + // Drain the buffer + //noinspection ResultOfMethodCallIgnored httpChunk.getContent().toString(); return clientResponse; } else { @@ -70,7 +73,7 @@ public RetryIfUnauthorizedResponseHandler( public ClientResponse> done(ClientResponse> clientResponse) { if (clientResponse.getObj().shouldRetry()) { - return ClientResponse.finished(RetryResponseHolder.retry()); + return ClientResponse.finished(RetryResponseHolder.retry()); } else { return wrap(httpResponseHandler.done(unwrap(clientResponse))); } @@ -85,9 +88,9 @@ public void exceptionCaught(ClientResponse> cl private ClientResponse> wrap(ClientResponse response) { if (response.isFinished()) { - return ClientResponse.finished(new RetryResponseHolder(false, response.getObj())); + return ClientResponse.finished(new RetryResponseHolder<>(false, response.getObj())); } else { - return ClientResponse.unfinished(new RetryResponseHolder(false, response.getObj())); + return ClientResponse.unfinished(new RetryResponseHolder<>(false, response.getObj())); } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryResponseHolder.java druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryResponseHolder.java index 2199e74acb..9b3eb7caaf 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryResponseHolder.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/security/RetryResponseHolder.java @@ -31,7 +31,7 @@ public RetryResponseHolder(boolean shouldRetry, T obj) public static RetryResponseHolder retry() { - return new RetryResponseHolder(true, null); + return new RetryResponseHolder<>(true, null); } public boolean shouldRetry() diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java index 611cad38ca..25e011dbe2 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java @@ -19,15 +19,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.hadoop.conf.Configuration; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapreduce.InputSplit; import java.io.IOException; import java.util.Map; @@ -35,23 +30,13 @@ /** * Record reader for results for Druid GroupByQuery. */ -public class DruidGroupByQueryRecordReader extends DruidQueryRecordReader { +public class DruidGroupByQueryRecordReader extends DruidQueryRecordReader { private final static TypeReference TYPE_REFERENCE = new TypeReference() { }; private MapBasedRow currentRow; private Map currentEvent; - @Override public void initialize(InputSplit split, Configuration conf) throws IOException { - super.initialize(split, conf); - } - - @Override public void initialize(InputSplit split, Configuration conf, ObjectMapper mapper, ObjectMapper smileMapper, - HttpClient httpClient - ) throws IOException { - super.initialize(split, conf, mapper, smileMapper, httpClient); - } - @Override protected JavaType getResultTypeDef() { return DruidStorageHandlerUtils.JSON_MAPPER.getTypeFactory().constructType(TYPE_REFERENCE); } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java index 8c10261a2e..fb3144980e 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java @@ -30,7 +30,6 @@ 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.hadoop.conf.Configuration; @@ -49,6 +48,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.Iterator; +import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -61,17 +61,12 @@ * DruidWritable containing the timestamp as well as all values resulting from * the query. */ -public abstract class DruidQueryRecordReader, R extends Comparable> +public abstract class DruidQueryRecordReader> extends RecordReader implements org.apache.hadoop.mapred.RecordReader { private static final Logger LOG = LoggerFactory.getLogger(DruidQueryRecordReader.class); - private HttpClient httpClient; - private ObjectMapper mapper; - // Smile mapper is used to read query results that are serialized as binary instead of json - private ObjectMapper smileMapper; - /** * Query that Druid executes. */ @@ -80,12 +75,7 @@ /** * Query results as a streaming iterator. */ - protected JsonParserIterator queryResultsIterator = null; - - /** - * Result type definition used to read the rows, this is query dependent. - */ - protected JavaType resultsType = null; + JsonParserIterator queryResultsIterator = null; @Override public void initialize(InputSplit split, TaskAttemptContext context) throws IOException { @@ -97,19 +87,23 @@ public void initialize(InputSplit split, Configuration conf, ObjectMapper mapper ) throws IOException { HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split; Preconditions.checkNotNull(hiveDruidSplit, "input split is null ???"); - this.mapper = Preconditions.checkNotNull(mapper, "object Mapper can not be null"); - // Smile mapper is used to read query results that are serilized as binary instead of json - this.smileMapper = Preconditions.checkNotNull(smileMapper, "Smile Mapper can not be null"); + ObjectMapper mapper1 = Preconditions.checkNotNull(mapper, "object Mapper can not be null"); + // Smile mapper is used to read query results that are serialized as binary instead of json + // Smile mapper is used to read query results that are serialized as binary instead of json + ObjectMapper smileMapper1 = Preconditions.checkNotNull(smileMapper, "Smile Mapper can not be null"); // Create query - this.query = this.mapper.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class); + this.query = mapper1.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class); Preconditions.checkNotNull(query); - this.resultsType = getResultTypeDef(); - this.httpClient = Preconditions.checkNotNull(httpClient, "need Http Client"); + /* + Result type definition used to read the rows, this is query dependent. + */ + JavaType resultsType = getResultTypeDef(); + HttpClient httpClient1 = Preconditions.checkNotNull(httpClient, "need Http Client"); final String[] locations = hiveDruidSplit.getLocations(); - boolean initlialized = false; + boolean initialized = false; int currentLocationIndex = 0; Exception ex = null; - while (!initlialized && currentLocationIndex < locations.length) { + while (!initialized && currentLocationIndex < locations.length) { String address = locations[currentLocationIndex++]; if(Strings.isNullOrEmpty(address)) { throw new IOException("can not fetch results from empty or null host value"); @@ -118,13 +112,14 @@ public void initialize(InputSplit split, Configuration conf, ObjectMapper mapper LOG.debug("Retrieving data from druid location[{}] using query:[{}] ", address, query); try { Request request = DruidStorageHandlerUtils.createSmileRequest(address, query); - Future inputStreamFuture = this.httpClient + Future inputStreamFuture = httpClient1 .go(request, new InputStreamResponseHandler()); - queryResultsIterator = new JsonParserIterator(this.smileMapper, resultsType, + //noinspection unchecked + queryResultsIterator = new JsonParserIterator(smileMapper1, resultsType, inputStreamFuture, request.getUrl().toString(), query ); queryResultsIterator.init(); - initlialized = true; + initialized = true; } catch (IOException | ExecutionException | InterruptedException e) { if(queryResultsIterator != null) { // We got exception while querying results from this host. @@ -143,13 +138,13 @@ public void initialize(InputSplit split, Configuration conf, ObjectMapper mapper } } - if(!initlialized) { + if(!initialized) { throw new RE( ex, "Failure getting results for query[%s] from locations[%s] because of [%s]", query, locations, - ex.getMessage() + Objects.requireNonNull(ex).getMessage() ); } } @@ -207,7 +202,7 @@ public void close() { * * @param druid Row type returned as result */ - protected class JsonParserIterator> implements Iterator, Closeable + @SuppressWarnings("TypeParameterHidesVisibleType") protected class JsonParserIterator> implements Iterator, Closeable { private JsonParser jp; private ObjectCodec objectCodec; @@ -224,12 +219,7 @@ public void close() { * @param url URL used to fetch the data, used mostly as message with exception stack to identify the faulty stream, thus this can be empty string. * @param query Query used to fetch the data, used mostly as message with exception stack, thus can be empty string. */ - public JsonParserIterator(ObjectMapper mapper, - JavaType typeRef, - Future future, - String url, - Query query - ) + JsonParserIterator(ObjectMapper mapper, JavaType typeRef, Future future, String url, Query query) { this.typeRef = typeRef; this.future = future; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java index 6c6514fea0..4b90aeb019 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidScanQueryRecordReader.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hive.druid.serde; -import io.druid.query.scan.ScanQuery; import io.druid.query.scan.ScanResultValue; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; @@ -35,14 +34,12 @@ * Record reader for results for Druid ScanQuery. */ public class DruidScanQueryRecordReader - extends DruidQueryRecordReader { + extends DruidQueryRecordReader { private static final TypeReference TYPE_REFERENCE = new TypeReference() { }; - private ScanResultValue current; - private Iterator> compactedValues = Iterators.emptyIterator(); @Override @@ -56,7 +53,7 @@ public boolean nextKeyValue() throws IOException { return true; } if (queryResultsIterator.hasNext()) { - current = queryResultsIterator.next(); + ScanResultValue current = queryResultsIterator.next(); compactedValues = ((List>) current.getEvents()).iterator(); return nextKeyValue(); } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java index 1ccca5f117..c0df682831 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java @@ -29,22 +29,19 @@ import io.druid.query.Result; import io.druid.query.select.EventHolder; -import io.druid.query.select.SelectQuery; import io.druid.query.select.SelectResultValue; /** * Record reader for results for Druid SelectQuery. */ public class DruidSelectQueryRecordReader - extends DruidQueryRecordReader> { + extends DruidQueryRecordReader> { private static final TypeReference> TYPE_REFERENCE = new TypeReference>() { }; - private Result current; - private Iterator values = Collections.emptyIterator(); @Override @@ -58,7 +55,7 @@ public boolean nextKeyValue() throws IOException { return true; } if (queryResultsIterator.hasNext()) { - current = queryResultsIterator.next(); + Result current = queryResultsIterator.next(); values = current.getValue().getEvents().iterator(); return nextKeyValue(); } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java index cf37e37c15..fbc8279dbb 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hive.druid.serde; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import io.druid.query.Druids; @@ -104,7 +103,7 @@ */ @SerDeSpec(schemaProps = { Constants.DRUID_DATA_SOURCE }) public class DruidSerDe extends AbstractSerDe { - protected static final Logger LOG = LoggerFactory.getLogger(DruidSerDe.class); + private static final Logger LOG = LoggerFactory.getLogger(DruidSerDe.class); private String[] columns; private PrimitiveTypeInfo[] types; @@ -179,32 +178,35 @@ private void initFromMetaDataQuery(final Configuration configuration, final Prop columnTypes.add(type instanceof TimestampLocalTZTypeInfo ? tsTZTypeInfo : type); inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type)); } - columns = columnNames.toArray(new String[columnNames.size()]); - types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]); + columns = columnNames.toArray(new String[0]); + types = columnTypes.toArray(new PrimitiveTypeInfo[0]); inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } private void initFromProperties(final Properties properties) throws SerDeException { - final List inspectors = new ArrayList<>(); - final List columnNames = new ArrayList<>(); - final List columnTypes = new ArrayList<>(); - columnNames.addAll(Utilities.getColumnNames(properties)); + final List columnNames = new ArrayList<>(Utilities.getColumnNames(properties)); if (!columnNames.contains(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN)) { throw new SerDeException("Timestamp column (' " + DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN + "') not specified in create table; list of columns is : " + properties .getProperty(serdeConstants.LIST_COLUMNS)); } - columnTypes.addAll(Lists.transform( - Lists.transform(Utilities.getColumnTypes(properties), type -> TypeInfoFactory.getPrimitiveTypeInfo(type)), - e -> e instanceof TimestampLocalTZTypeInfo ? tsTZTypeInfo : e - )); - inspectors.addAll(Lists.transform(columnTypes, - (Function) type -> PrimitiveObjectInspectorFactory - .getPrimitiveWritableObjectInspector(type) - )); - columns = columnNames.toArray(new String[columnNames.size()]); - types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]); + final List + columnTypes = + Utilities.getColumnTypes(properties) + .stream() + .map(TypeInfoFactory::getPrimitiveTypeInfo) + .collect(Collectors.toList()) + .stream() + .map(e -> e instanceof TimestampLocalTZTypeInfo ? tsTZTypeInfo : e) + .collect(Collectors.toList()); + final List + inspectors = + columnTypes.stream() + .map(PrimitiveObjectInspectorFactory::getPrimitiveWritableObjectInspector) + .collect(Collectors.toList()); + columns = columnNames.toArray(new String[0]); + types = columnTypes.toArray(new PrimitiveTypeInfo[0]); inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } @@ -223,8 +225,8 @@ private void initFromDruidQueryPlan(Properties properties, String druidQuery) { // LIMIT 1 // is planed in a way where we only push a filter down and keep the project of null as hive project. Thus empty // columns - columnNames = Collections.EMPTY_LIST; - columnTypes = Collections.EMPTY_LIST; + columnNames = Collections.emptyList(); + columnTypes = Collections.emptyList(); } else { columnNames = Arrays.stream(fieldNamesProperty.trim().split(",")).collect(Collectors.toList()); columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(fieldTypesProperty).stream() diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java index c04f2dcbe1..aead194479 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java @@ -25,18 +25,18 @@ /** * Utils class for Druid SerDe. */ -public final class DruidSerDeUtils { +final class DruidSerDeUtils { private static final Logger LOG = LoggerFactory.getLogger(DruidSerDeUtils.class); - protected static final String FLOAT_TYPE = "FLOAT"; - protected static final String DOUBLE_TYPE = "DOUBLE"; - protected static final String LONG_TYPE = "LONG"; - protected static final String STRING_TYPE = "STRING"; + private static final String FLOAT_TYPE = "FLOAT"; + private static final String DOUBLE_TYPE = "DOUBLE"; + private static final String LONG_TYPE = "LONG"; + private static final String STRING_TYPE = "STRING"; /* This method converts from the String representation of Druid type * to the corresponding Hive type */ - public static PrimitiveTypeInfo convertDruidToHiveType(String typeName) { + static PrimitiveTypeInfo convertDruidToHiveType(String typeName) { typeName = typeName.toUpperCase(); switch (typeName) { case FLOAT_TYPE: diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java index fcda3efd34..41fd341341 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java @@ -20,7 +20,6 @@ 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.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.io.NullWritable; @@ -31,7 +30,7 @@ * Record reader for results for Druid TimeseriesQuery. */ public class DruidTimeseriesQueryRecordReader - extends DruidQueryRecordReader> { + extends DruidQueryRecordReader> { private static final TypeReference TYPE_REFERENCE = new TypeReference>() { }; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java index e24bfff495..b54aa4aafa 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java @@ -18,28 +18,24 @@ package org.apache.hadoop.hive.druid.serde; import java.io.IOException; -import java.io.InputStream; import java.util.Collections; import java.util.Iterator; -import java.util.List; import com.fasterxml.jackson.databind.JavaType; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; import org.apache.hadoop.io.NullWritable; 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; /** * Record reader for results for Druid TopNQuery. */ public class DruidTopNQueryRecordReader - extends DruidQueryRecordReader> { + extends DruidQueryRecordReader> { private static final TypeReference> TYPE_REFERENCE = new TypeReference>() { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java index 7390647c4b..1a3f06ce0c 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java @@ -78,19 +78,19 @@ public DruidWritable(List value){ return value; } - public List getCompactedValue() { + List getCompactedValue() { if(!compacted){ throw new UnsupportedOperationException("non compacted DruidWritable does not support getCompactedValue(), use getValue()"); } return compactedValue; } - public void setCompactedValue(List compactedValue) + void setCompactedValue(List compactedValue) { this.compactedValue = compactedValue; } - public boolean isCompacted(){ + boolean isCompacted(){ return compacted; } diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java index 627f0787cb..bf42a74f0f 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java @@ -18,13 +18,11 @@ package org.apache.hadoop.hive.druid; -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.junit.Assert; import org.junit.rules.ExternalResource; import org.skife.jdbi.v2.DBI; @@ -32,6 +30,7 @@ import java.sql.SQLException; import java.util.UUID; +import java.util.function.Supplier; public class DerbyConnectorTestUtility extends DerbyConnector { private final String jdbcUri; @@ -48,7 +47,7 @@ protected DerbyConnectorTestUtility( Supplier dbTables, String jdbcUri ) { - super(new DerbyMetadataStorage(config.get()), config, dbTables, new DBI(jdbcUri + ";create=true")); + super(new DerbyMetadataStorage(config.get()), config::get, dbTables::get, new DBI(jdbcUri + ";create=true")); this.jdbcUri = jdbcUri; } @@ -86,7 +85,7 @@ public DerbyConnectorRule() { private DerbyConnectorRule( final String defaultBase ) { - this(Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(defaultBase))); + this(Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(defaultBase))::get); } public DerbyConnectorRule( @@ -103,7 +102,7 @@ public String getConnectURI() { @Override protected void before() throws Throwable { - connector = new DerbyConnectorTestUtility(Suppliers.ofInstance(connectorConfig), dbTables); + connector = new DerbyConnectorTestUtility(Suppliers.ofInstance(connectorConfig)::get, dbTables); connector.getDBI().open().close(); // create db } @@ -116,10 +115,6 @@ public DerbyConnectorTestUtility getConnector() { return connector; } - public MetadataStorageConnectorConfig getMetadataConnectorConfig() { - return connectorConfig; - } - public Supplier metadataTablesConfigSupplier() { return dbTables; } diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java index 510330d5d0..0cb3c237b1 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java @@ -18,6 +18,10 @@ package org.apache.hadoop.hive.druid; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import io.druid.indexer.JobHelper; import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler; import io.druid.metadata.MetadataStorageTablesConfig; @@ -29,7 +33,6 @@ import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; - import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -39,12 +42,6 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; - -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.DateTimeZone; import org.joda.time.Interval; import org.junit.After; @@ -55,16 +52,12 @@ import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; import java.io.File; import java.io.IOException; import java.io.OutputStream; -import java.sql.ResultSet; -import java.sql.SQLException; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.UUID; @@ -96,10 +89,9 @@ private DataSegment createSegment(String location) throws IOException { private DataSegment createSegment(String location, Interval interval, String version, ShardSpec shardSpec) throws IOException { FileUtils.writeStringToFile(new File(location), "dummySegmentData"); - DataSegment dataSegment = DataSegment.builder().dataSource(DATA_SOURCE_NAME).version(version) + return DataSegment.builder().dataSource(DATA_SOURCE_NAME).version(version) .interval(interval).shardSpec(shardSpec) .loadSpec(ImmutableMap.of("path", location)).build(); - return dataSegment; } @Before @@ -133,7 +125,7 @@ public void tearDown() { temporaryFolder.delete(); } - Table tableMock = Mockito.mock(Table.class); + private final Table tableMock = Mockito.mock(Table.class); @Test public void testPreCreateTableWillCreateSegmentsTable() throws MetaException { @@ -160,7 +152,7 @@ public void testPreCreateTableWhenDataSourceExists() throws MetaException, IOExc Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); DataSegment dataSegment = createSegment(new Path(taskDirPath, "intermediatePath").toString()); - sqlMetadataStorageUpdaterJobHandler.publishSegments(segmentsTable, Arrays.asList(dataSegment), + sqlMetadataStorageUpdaterJobHandler.publishSegments(segmentsTable, Collections.singletonList(dataSegment), DruidStorageHandlerUtils.JSON_MAPPER ); @@ -291,7 +283,7 @@ public void testDeleteSegment() throws IOException, SegmentLoadingException { ); Path indexPath = new Path(segmentOutputPath, "index.zip"); DataSegment dataSegmentWithLoadspect = DataSegment.builder(dataSegment).loadSpec( - ImmutableMap.of("path", indexPath)).build(); + ImmutableMap.of("path", indexPath)).build(); OutputStream outputStream = localFileSystem.create(indexPath, true); outputStream.close(); Assert.assertTrue("index file is not created ??", localFileSystem.exists(indexPath)); @@ -328,9 +320,11 @@ public void testCommitInsertOverwriteTable() throws MetaException, IOException { DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); // This create and publish the segment to be overwritten - List existingSegments = Arrays - .asList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new LinearShardSpec(0))); + List existingSegments = + Collections.singletonList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0))); DruidStorageHandlerUtils .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, existingSegments, @@ -381,9 +375,11 @@ public void testCommitMultiInsertOverwriteTable() throws MetaException, IOExcept DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); // This create and publish the segment to be overwritten - List existingSegments = Arrays - .asList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new LinearShardSpec(0))); + List existingSegments = + Collections.singletonList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0))); DruidStorageHandlerUtils .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, existingSegments, @@ -486,31 +482,20 @@ public void testCommitMultiInsertOverwriteTable() throws MetaException, IOExcept private List getUsedSegmentsList(DerbyConnectorTestUtility connector, final MetadataStorageTablesConfig metadataStorageTablesConfig) { return connector.getDBI() - .withHandle(new HandleCallback>() { - @Override - public List withHandle(Handle handle) throws Exception { - return handle - .createQuery(String.format( - "SELECT payload FROM %s WHERE used=true ORDER BY created_date ASC", - metadataStorageTablesConfig.getSegmentsTable())) - .map(new ResultSetMapper() { - - @Override - public DataSegment map(int i, ResultSet resultSet, - StatementContext statementContext) - throws SQLException { - try { - return DruidStorageHandlerUtils.JSON_MAPPER.readValue( - resultSet.getBytes("payload"), - DataSegment.class - ); - } catch (IOException e) { - throw Throwables.propagate(e); - } - } - }).list(); - } - }); + .withHandle(handle -> handle + .createQuery(String.format( + "SELECT payload FROM %s WHERE used=true ORDER BY created_date ASC", + metadataStorageTablesConfig.getSegmentsTable())) + .map((i, resultSet, statementContext) -> { + try { + return DruidStorageHandlerUtils.JSON_MAPPER.readValue( + resultSet.getBytes("payload"), + DataSegment.class + ); + } catch (IOException e) { + throw Throwables.propagate(e); + } + }).list()); } @Test @@ -521,9 +506,11 @@ public void testCommitInsertIntoTable() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); - List existingSegments = Arrays - .asList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new LinearShardSpec(1))); + List existingSegments = + Collections.singletonList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(1))); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(config.get(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY))); DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); @@ -578,9 +565,11 @@ public void testInsertIntoAppendOneMorePartition() throws MetaException, IOExcep pusherConfig.setStorageDirectory(config.get(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY))); DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); - List existingSegments = Arrays - .asList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new LinearShardSpec(0))); + List existingSegments = + Collections.singletonList(createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0))); DruidStorageHandlerUtils .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, existingSegments, @@ -628,9 +617,11 @@ public void testCommitInsertIntoWhenDestinationSegmentFileExist() druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); - List existingSegments = Arrays - .asList(createSegment(new Path(taskDirPath, "index_old.zip").toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new LinearShardSpec(1))); + List existingSegments = + Collections.singletonList(createSegment(new Path(taskDirPath, "index_old.zip").toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(1))); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(config.get(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY))); DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); @@ -732,7 +723,7 @@ public void testCommitInsertIntoWithNonExtendableSegment() throws MetaException, Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); List existingSegments = Arrays .asList(createSegment(new Path(taskDirPath, "index_old_1.zip").toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new NoneShardSpec()), + new Interval(100, 150, DateTimeZone.UTC), "v0", NoneShardSpec.instance()), createSegment(new Path(taskDirPath, "index_old_2.zip").toString(), new Interval(200, 250, DateTimeZone.UTC), "v0", new LinearShardSpec(0)), createSegment(new Path(taskDirPath, "index_old_3.zip").toString(), diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java index 7da3a307db..513119ea32 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java @@ -26,12 +26,11 @@ import org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat; import org.apache.hadoop.hive.druid.io.HiveDruidSplit; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.junit.Test; import io.druid.query.Query; import junit.framework.TestCase; -public class TestHiveDruidQueryBasedInputFormat extends TestCase { +@SuppressWarnings("SameParameterValue") public class TestHiveDruidQueryBasedInputFormat extends TestCase { private static final String TIMESERIES_QUERY = "{ \"queryType\": \"timeseries\", " @@ -151,7 +150,6 @@ + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":5,\"fromNext\":false}," + "\"context\":{\"druid.query.fetch\":true,\"queryId\":\"\"}}, [localhost:8082]}]"; - @Test public void testTimeZone() throws Exception { DruidQueryBasedInputFormat input = new DruidQueryBasedInputFormat(); diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java index acde2394d7..06fcfd1e72 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/serde/TestDruidSerDe.java @@ -33,11 +33,11 @@ import java.util.List; import java.util.Map.Entry; import java.util.Properties; +import java.util.stream.Collectors; 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.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveChar; @@ -77,12 +77,8 @@ import org.junit.Rule; import org.junit.Test; -import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.SettableFuture; import io.druid.data.input.Row; @@ -97,52 +93,54 @@ * Basic tests for Druid SerDe. The examples are taken from Druid 0.9.1.1 * documentation. */ -public class TestDruidSerDe { +@SuppressWarnings({ "SameParameterValue", "SpellCheckingInspection" }) public class TestDruidSerDe { // Timeseries query - private static final String TIMESERIES_QUERY = - "{ \"queryType\": \"timeseries\", " - + " \"dataSource\": \"sample_datasource\", " - + " \"granularity\": \"day\", " - + " \"descending\": \"true\", " - + " \"filter\": { " - + " \"type\": \"and\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"sample_dimension1\", \"value\": \"sample_value1\" }, " - + " { \"type\": \"or\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"sample_dimension2\", \"value\": \"sample_value2\" }, " - + " { \"type\": \"selector\", \"dimension\": \"sample_dimension3\", \"value\": \"sample_value3\" } " - + " ] " - + " } " - + " ] " - + " }, " - + " \"aggregations\": [ " - + " { \"type\": \"longSum\", \"name\": \"sample_name1\", \"fieldName\": \"sample_fieldName1\" }, " - + " { \"type\": \"doubleSum\", \"name\": \"sample_name2\", \"fieldName\": \"sample_fieldName2\" } " - + " ], " - + " \"postAggregations\": [ " - + " { \"type\": \"arithmetic\", " - + " \"name\": \"sample_divide\", " - + " \"fn\": \"/\", " - + " \"fields\": [ " - + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" }, " - + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" } " - + " ] " - + " } " - + " ], " - + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}"; + private static final String + TIMESERIES_QUERY = + "{ \"queryType\": \"timeseries\", " + + " \"dataSource\": \"sample_datasource\", " + + " \"granularity\": \"day\", " + + " \"descending\": \"true\", " + + " \"filter\": { " + + " \"type\": \"and\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"sample_dimension1\", \"value\": \"sample_value1\" }, " + + " { \"type\": \"or\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"sample_dimension2\", \"value\": \"sample_value2\" }, " + + " { \"type\": \"selector\", \"dimension\": \"sample_dimension3\", \"value\": \"sample_value3\" } " + + " ] " + + " } " + + " ] " + + " }, " + + " \"aggregations\": [ " + + " { \"type\": \"longSum\", \"name\": \"sample_name1\", \"fieldName\": \"sample_fieldName1\" }, " + + " { \"type\": \"doubleSum\", \"name\": \"sample_name2\", \"fieldName\": \"sample_fieldName2\" } " + + " ], " + + " \"postAggregations\": [ " + + " { \"type\": \"arithmetic\", " + + " \"name\": \"sample_divide\", " + + " \"fn\": \"/\", " + + " \"fields\": [ " + + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" }, " + + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" } " + + " ] " + + " } " + + " ], " + + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}"; // Timeseries query results - private static final String TIMESERIES_QUERY_RESULTS = - "[ " - + "{ " - + " \"timestamp\": \"2012-01-01T00:00:00.000Z\", " - + " \"result\": { \"sample_name1\": 0, \"sample_name2\": 1.0, \"sample_divide\": 2.2222 } " - + "}, " - + "{ " - + " \"timestamp\": \"2012-01-02T00:00:00.000Z\", " - + " \"result\": { \"sample_name1\": 2, \"sample_name2\": 3.32, \"sample_divide\": 4 } " - + "}]"; + private static final String + TIMESERIES_QUERY_RESULTS = + "[ " + + "{ " + + " \"timestamp\": \"2012-01-01T00:00:00.000Z\", " + + " \"result\": { \"sample_name1\": 0, \"sample_name2\": 1.0, \"sample_divide\": 2.2222 } " + + "}, " + + "{ " + + " \"timestamp\": \"2012-01-02T00:00:00.000Z\", " + + " \"result\": { \"sample_name1\": 2, \"sample_name2\": 3.32, \"sample_divide\": 4 } " + + "}]"; private byte[] tsQueryResults; private byte[] topNQueryResults; @@ -152,209 +150,215 @@ private byte[] groupByMonthExtractQueryResults; private byte[] scanQueryResults; - // Timeseries query results as records - private static final Object[][] TIMESERIES_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), new LongWritable(0), - new FloatWritable(1.0F), new FloatWritable(2.2222F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325462400000L).atZone(ZoneOffset.UTC))), new LongWritable(2), - new FloatWritable(3.32F), new FloatWritable(4F) } - }; + private static final Object[][] + TIMESERIES_QUERY_RESULTS_RECORDS = + new Object[][] { new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), + new LongWritable(0), new FloatWritable(1.0F), new FloatWritable(2.2222F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325462400000L).atZone(ZoneOffset.UTC))), + new LongWritable(2), new FloatWritable(3.32F), new FloatWritable(4F) } }; // Timeseries query results as records (types defined by metastore) private static final String TIMESERIES_COLUMN_NAMES = "timestamp,sample_name1,sample_name2,sample_divide"; private static final String TIMESERIES_COLUMN_TYPES = "timestamp with local time zone,bigint,float,float"; // TopN query - private static final String TOPN_QUERY = - "{ \"queryType\": \"topN\", " - + " \"dataSource\": \"sample_data\", " - + " \"dimension\": \"sample_dim\", " - + " \"threshold\": 5, " - + " \"metric\": \"count\", " - + " \"granularity\": \"all\", " - + " \"filter\": { " - + " \"type\": \"and\", " - + " \"fields\": [ " - + " { " - + " \"type\": \"selector\", " - + " \"dimension\": \"dim1\", " - + " \"value\": \"some_value\" " - + " }, " - + " { " - + " \"type\": \"selector\", " - + " \"dimension\": \"dim2\", " - + " \"value\": \"some_other_val\" " - + " } " - + " ] " - + " }, " - + " \"aggregations\": [ " - + " { " - + " \"type\": \"longSum\", " - + " \"name\": \"count\", " - + " \"fieldName\": \"count\" " - + " }, " - + " { " - + " \"type\": \"doubleSum\", " - + " \"name\": \"some_metric\", " - + " \"fieldName\": \"some_metric\" " - + " } " - + " ], " - + " \"postAggregations\": [ " - + " { " - + " \"type\": \"arithmetic\", " - + " \"name\": \"sample_divide\", " - + " \"fn\": \"/\", " - + " \"fields\": [ " - + " { " - + " \"type\": \"fieldAccess\", " - + " \"name\": \"some_metric\", " - + " \"fieldName\": \"some_metric\" " - + " }, " - + " { " - + " \"type\": \"fieldAccess\", " - + " \"name\": \"count\", " - + " \"fieldName\": \"count\" " - + " } " - + " ] " - + " } " - + " ], " - + " \"intervals\": [ " - + " \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" " - + " ]}"; + private static final String + TOPN_QUERY = + "{ \"queryType\": \"topN\", " + + " \"dataSource\": \"sample_data\", " + + " \"dimension\": \"sample_dim\", " + + " \"threshold\": 5, " + + " \"metric\": \"count\", " + + " \"granularity\": \"all\", " + + " \"filter\": { " + + " \"type\": \"and\", " + + " \"fields\": [ " + + " { " + + " \"type\": \"selector\", " + + " \"dimension\": \"dim1\", " + + " \"value\": \"some_value\" " + + " }, " + + " { " + + " \"type\": \"selector\", " + + " \"dimension\": \"dim2\", " + + " \"value\": \"some_other_val\" " + + " } " + + " ] " + + " }, " + + " \"aggregations\": [ " + + " { " + + " \"type\": \"longSum\", " + + " \"name\": \"count\", " + + " \"fieldName\": \"count\" " + + " }, " + + " { " + + " \"type\": \"doubleSum\", " + + " \"name\": \"some_metric\", " + + " \"fieldName\": \"some_metric\" " + + " } " + + " ], " + + " \"postAggregations\": [ " + + " { " + + " \"type\": \"arithmetic\", " + + " \"name\": \"sample_divide\", " + + " \"fn\": \"/\", " + + " \"fields\": [ " + + " { " + + " \"type\": \"fieldAccess\", " + + " \"name\": \"some_metric\", " + + " \"fieldName\": \"some_metric\" " + + " }, " + + " { " + + " \"type\": \"fieldAccess\", " + + " \"name\": \"count\", " + + " \"fieldName\": \"count\" " + + " } " + + " ] " + + " } " + + " ], " + + " \"intervals\": [ " + + " \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" " + + " ]}"; // TopN query results - private static final String TOPN_QUERY_RESULTS = - "[ " - + " { " - + " \"timestamp\": \"2013-08-31T00:00:00.000Z\", " - + " \"result\": [ " - + " { " - + " \"sample_dim\": \"dim1_val\", " - + " \"count\": 111, " - + " \"some_metric\": 10669, " - + " \"sample_divide\": 96.11711711711712 " - + " }, " - + " { " - + " \"sample_dim\": \"another_dim1_val\", " - + " \"count\": 88, " - + " \"some_metric\": 28344, " - + " \"sample_divide\": 322.09090909090907 " - + " }, " - + " { " - + " \"sample_dim\": \"dim1_val3\", " - + " \"count\": 70, " - + " \"some_metric\": 871, " - + " \"sample_divide\": 12.442857142857143 " - + " }, " - + " { " - + " \"sample_dim\": \"dim1_val4\", " - + " \"count\": 62, " - + " \"some_metric\": 815, " - + " \"sample_divide\": 13.14516129032258 " - + " }, " - + " { " - + " \"sample_dim\": \"dim1_val5\", " - + " \"count\": 60, " - + " \"some_metric\": 2787, " - + " \"sample_divide\": 46.45 " - + " } " - + " ] " - + " }]"; + private static final String + TOPN_QUERY_RESULTS = + "[ " + + " { " + + " \"timestamp\": \"2013-08-31T00:00:00.000Z\", " + + " \"result\": [ " + + " { " + + " \"sample_dim\": \"dim1_val\", " + + " \"count\": 111, " + + " \"some_metric\": 10669, " + + " \"sample_divide\": 96.11711711711712 " + + " }, " + + " { " + + " \"sample_dim\": \"another_dim1_val\", " + + " \"count\": 88, " + + " \"some_metric\": 28344, " + + " \"sample_divide\": 322.09090909090907 " + + " }, " + + " { " + + " \"sample_dim\": \"dim1_val3\", " + + " \"count\": 70, " + + " \"some_metric\": 871, " + + " \"sample_divide\": 12.442857142857143 " + + " }, " + + " { " + + " \"sample_dim\": \"dim1_val4\", " + + " \"count\": 62, " + + " \"some_metric\": 815, " + + " \"sample_divide\": 13.14516129032258 " + + " }, " + + " { " + + " \"sample_dim\": \"dim1_val5\", " + + " \"count\": 60, " + + " \"some_metric\": 2787, " + + " \"sample_divide\": 46.45 " + + " } " + + " ] " + + " }]"; // TopN query results as records - private static final Object[][] TOPN_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("dim1_val"), new LongWritable(111), new FloatWritable(10669F), - new FloatWritable(96.11711711711712F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("another_dim1_val"), new LongWritable(88), new FloatWritable(28344F), - new FloatWritable(322.09090909090907F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("dim1_val3"), new LongWritable(70), new FloatWritable(871F), - new FloatWritable(12.442857142857143F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("dim1_val4"), new LongWritable(62), new FloatWritable(815F), - new FloatWritable(13.14516129032258F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("dim1_val5"), new LongWritable(60), new FloatWritable(2787F), - new FloatWritable(46.45F) } - }; + private static final Object[][] + TOPN_QUERY_RESULTS_RECORDS = + new Object[][] { new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), + new Text("dim1_val"), new LongWritable(111), new FloatWritable(10669F), + new FloatWritable(96.11711711711712F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), + new Text("another_dim1_val"), new LongWritable(88), new FloatWritable(28344F), + new FloatWritable(322.09090909090907F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), + new Text("dim1_val3"), new LongWritable(70), new FloatWritable(871F), + new FloatWritable(12.442857142857143F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), + new Text("dim1_val4"), new LongWritable(62), new FloatWritable(815F), new FloatWritable(13.14516129032258F) }, + new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L) + .atZone(ZoneOffset.UTC))), new Text("dim1_val5"), new LongWritable(60), new FloatWritable(2787F), + new FloatWritable(46.45F) } }; // TopN query results as records (types defined by metastore) private static final String TOPN_COLUMN_NAMES = "timestamp,sample_dim,count,some_metric,sample_divide"; private static final String TOPN_COLUMN_TYPES = "timestamp with local time zone,string,bigint,float,float"; // GroupBy query - private static final String GROUP_BY_QUERY = - "{ " - + " \"queryType\": \"groupBy\", " - + " \"dataSource\": \"sample_datasource\", " - + " \"granularity\": \"day\", " - + " \"dimensions\": [\"country\", \"device\"], " - + " \"limitSpec\": {" - + " \"type\": \"default\"," - + " \"limit\": 5000," - + " \"columns\": [\"country\", \"data_transfer\"] }, " - + " \"filter\": { " - + " \"type\": \"and\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"carrier\", \"value\": \"AT&T\" }, " - + " { \"type\": \"or\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Apple\" }, " - + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Samsung\" } " - + " ] " - + " } " - + " ] " - + " }, " - + " \"aggregations\": [ " - + " { \"type\": \"longSum\", \"name\": \"total_usage\", \"fieldName\": \"user_count\" }, " - + " { \"type\": \"doubleSum\", \"name\": \"data_transfer\", \"fieldName\": \"data_transfer\" } " - + " ], " - + " \"postAggregations\": [ " - + " { \"type\": \"arithmetic\", " - + " \"name\": \"avg_usage\", " - + " \"fn\": \"/\", " - + " \"fields\": [ " - + " { \"type\": \"fieldAccess\", \"fieldName\": \"data_transfer\" }, " - + " { \"type\": \"fieldAccess\", \"fieldName\": \"total_usage\" } " - + " ] " - + " } " - + " ], " - + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ], " - + " \"having\": { " - + " \"type\": \"greaterThan\", " - + " \"aggregation\": \"total_usage\", " - + " \"value\": 100 " - + " }}"; + private static final String + GROUP_BY_QUERY = + "{ " + + " \"queryType\": \"groupBy\", " + + " \"dataSource\": \"sample_datasource\", " + + " \"granularity\": \"day\", " + + " \"dimensions\": [\"country\", \"device\"], " + + " \"limitSpec\": {" + + " \"type\": \"default\"," + + " \"limit\": 5000," + + " \"columns\": [\"country\", \"data_transfer\"] }, " + + " \"filter\": { " + + " \"type\": \"and\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"carrier\", \"value\": \"AT&T\" }, " + + " { \"type\": \"or\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Apple\" }, " + + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Samsung\" } " + + " ] " + + " } " + + " ] " + + " }, " + + " \"aggregations\": [ " + + " { \"type\": \"longSum\", \"name\": \"total_usage\", \"fieldName\": \"user_count\" }, " + + " { \"type\": \"doubleSum\", \"name\": \"data_transfer\", \"fieldName\": \"data_transfer\" } " + + " ], " + + " \"postAggregations\": [ " + + " { \"type\": \"arithmetic\", " + + " \"name\": \"avg_usage\", " + + " \"fn\": \"/\", " + + " \"fields\": [ " + + " { \"type\": \"fieldAccess\", \"fieldName\": \"data_transfer\" }, " + + " { \"type\": \"fieldAccess\", \"fieldName\": \"total_usage\" } " + + " ] " + + " } " + + " ], " + + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ], " + + " \"having\": { " + + " \"type\": \"greaterThan\", " + + " \"aggregation\": \"total_usage\", " + + " \"value\": 100 " + + " }}"; // GroupBy query results - private static final String GROUP_BY_QUERY_RESULTS = - "[ " - + " { " - + " \"version\" : \"v1\", " - + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", " - + " \"event\" : { " - + " \"country\" : \"India\", " - + " \"device\" : \"phone\", " - + " \"total_usage\" : 88, " - + " \"data_transfer\" : 29.91233453, " - + " \"avg_usage\" : 60.32 " - + " } " - + " }, " - + " { " - + " \"version\" : \"v1\", " - + " \"timestamp\" : \"2012-01-01T00:00:12.000Z\", " - + " \"event\" : { " - + " \"country\" : \"Spain\", " - + " \"device\" : \"pc\", " - + " \"total_usage\" : 16, " - + " \"data_transfer\" : 172.93494959, " - + " \"avg_usage\" : 6.333333 " - + " } " - + " }]"; - - private static final String GB_TIME_EXTRACTIONS = "{\"queryType\":\"groupBy\",\"dataSource\":\"sample_datasource\"," + private static final String + GROUP_BY_QUERY_RESULTS = + "[ " + + " { " + + " \"version\" : \"v1\", " + + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", " + + " \"event\" : { " + + " \"country\" : \"India\", " + + " \"device\" : \"phone\", " + + " \"total_usage\" : 88, " + + " \"data_transfer\" : 29.91233453, " + + " \"avg_usage\" : 60.32 " + + " } " + + " }, " + + " { " + + " \"version\" : \"v1\", " + + " \"timestamp\" : \"2012-01-01T00:00:12.000Z\", " + + " \"event\" : { " + + " \"country\" : \"Spain\", " + + " \"device\" : \"pc\", " + + " \"total_usage\" : 16, " + + " \"data_transfer\" : 172.93494959, " + + " \"avg_usage\" : 6.333333 " + + " } " + + " }]"; + + private static final String + GB_TIME_EXTRACTIONS = + "{\"queryType\":\"groupBy\",\"dataSource\":\"sample_datasource\"," + "\"granularity\":\"all\",\"dimensions\":" + "[{\"type\":\"extraction\",\"dimension\":\"__time\",\"outputName\":\"extract\",\"extractionFn\":" + "{\"type\":\"timeFormat\",\"format\":\"yyyy-MM-dd'T'HH:mm:ss.SSS'Z'\",\"timeZone\":\"UTC\"}}]," @@ -362,7 +366,9 @@ + "\"aggregations\":[{\"type\":\"count\",\"name\":\"$f1\"}]," + "\"intervals\":[\"1900-01-01T00:00:00.000/3000-01-01T00:00:00.000\"]}"; - private static final String GB_TIME_EXTRACTIONS_RESULTS = "[ " + private static final String + GB_TIME_EXTRACTIONS_RESULTS = + "[ " + " { " + " \"version\" : \"v1\", " + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", " @@ -380,7 +386,9 @@ + " } " + " }]"; - private static final String GB_MONTH_EXTRACTIONS_RESULTS = "[ " + private static final String + GB_MONTH_EXTRACTIONS_RESULTS = + "[ " + " { " + " \"version\" : \"v1\", " + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", " @@ -398,8 +406,9 @@ + " } " + " }]"; - - private final static String GB_MONTH_EXTRACTIONS = "{\"queryType\":\"groupBy\",\"dataSource\":\"sample_datasource\"," + private final static String + GB_MONTH_EXTRACTIONS = + "{\"queryType\":\"groupBy\",\"dataSource\":\"sample_datasource\"," + "\"granularity\":\"all\"," + "\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"__time\",\"outputName\":\"extract_month\"," + "\"extractionFn\":{\"type\":\"timeFormat\",\"format\":\"M\",\"timeZone\":\"UTC\",\"locale\":\"en-US\"}}]," @@ -407,200 +416,210 @@ + "\"intervals\":[\"1900-01-01T00:00:00.000/3000-01-01T00:00:00.000\"]}"; // GroupBy query results as records - private static final Object[][] GROUP_BY_QUERY_EXTRACTION_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), - new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), - new LongWritable(200) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), - new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), - new LongWritable(400) } - }; - - private static final Object[][] GROUP_BY_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), new Text("India"), - new Text("phone"), new LongWritable(88), new DoubleWritable(29.91233453), - new FloatWritable(60.32F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), new Text("Spain"), - new Text("pc"), new LongWritable(16), new DoubleWritable(172.93494959), - new FloatWritable(6.333333F) } - }; - - private static final Object[][] GB_MONTH_EXTRACTION_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), - new IntWritable(1), - new LongWritable(200) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), new IntWritable(1), - new LongWritable(400) } - }; + private static final Object[][] + GROUP_BY_QUERY_EXTRACTION_RESULTS_RECORDS = + new Object[][] { new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), + new LongWritable(200) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), + new LongWritable(400) } }; + + private static final Object[][] + GROUP_BY_QUERY_RESULTS_RECORDS = + new Object[][] { new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), + new Text("India"), new Text("phone"), new LongWritable(88), new DoubleWritable(29.91233453), + new FloatWritable(60.32F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), + new Text("Spain"), new Text("pc"), new LongWritable(16), new DoubleWritable(172.93494959), + new FloatWritable(6.333333F) } }; + + private static final Object[][] + GB_MONTH_EXTRACTION_RESULTS_RECORDS = + new Object[][] { new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376000000L).atZone(ZoneOffset.UTC))), + new IntWritable(1), new LongWritable(200) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), + new IntWritable(1), new LongWritable(400) } }; // GroupBy query results as records (types defined by metastore) private static final String GROUP_BY_COLUMN_NAMES = "timestamp,country,device,total_usage,data_transfer,avg_usage"; - private static final String GROUP_BY_COLUMN_TYPES = "timestamp with local time zone,string,string,bigint,double,float"; + private static final String + GROUP_BY_COLUMN_TYPES = + "timestamp with local time zone,string,string,bigint,double,float"; private static final String GB_TIME_EXTRACTIONS_COLUMN_NAMES = "timestamp,extract,$f1"; - private static final String GB_TIME_EXTRACTIONS_COLUMN_TYPES = "timestamp with local time zone,timestamp with local time zone,bigint"; + private static final String + GB_TIME_EXTRACTIONS_COLUMN_TYPES = + "timestamp with local time zone,timestamp with local time zone,bigint"; private static final String GB_MONTH_EXTRACTIONS_COLUMN_NAMES = "timestamp,extract_month,$f1"; private static final String GB_MONTH_EXTRACTIONS_COLUMN_TYPES = "timestamp with local time zone,int,bigint"; // Select query - private static final String SELECT_QUERY = - "{ \"queryType\": \"select\", " - + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " - + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], " - + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " - + " \"granularity\": \"all\", " - + " \"intervals\": [ \"2013-01-01/2013-01-02\" ], " - + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5} }"; + private static final String + SELECT_QUERY = + "{ \"queryType\": \"select\", " + + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " + + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], " + + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " + + " \"granularity\": \"all\", " + + " \"intervals\": [ \"2013-01-01/2013-01-02\" ], " + + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5} }"; // Select query results - private static final String SELECT_QUERY_RESULTS = - "[{ " - + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " - + " \"result\" : { " - + " \"pagingIdentifiers\" : { " - + " \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\" : 4 }, " - + " \"events\" : [ { " - + " \"segmentId\" : \"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 0, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " - + " \"robot\" : 1, " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"11._korpus_(NOVJ)\", " - + " \"language\" : \"sl\", " - + " \"newpage\" : \"0\", " - + " \"user\" : \"EmausBot\", " - + " \"count\" : 1.0, " - + " \"added\" : 39.0, " - + " \"delta\" : 39.0, " - + " \"variation\" : 39.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 1, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " - + " \"robot\" : 0, " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"112_U.S._580\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 70.0, " - + " \"delta\" : 70.0, " - + " \"variation\" : 70.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 2, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " - + " \"robot\" : 0, " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"113_U.S._243\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 77.0, " - + " \"delta\" : 77.0, " - + " \"variation\" : 77.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 3, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " - + " \"robot\" : 0, " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"113_U.S._73\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 70.0, " - + " \"delta\" : 70.0, " - + " \"variation\" : 70.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 4, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " - + " \"robot\" : 0, " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"113_U.S._756\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 68.0, " - + " \"delta\" : 68.0, " - + " \"variation\" : 68.0, " - + " \"deleted\" : 0.0 " - + " } " - + " } ] }} ]"; + private static final String + SELECT_QUERY_RESULTS = + "[{ " + + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " + + " \"result\" : { " + + " \"pagingIdentifiers\" : { " + + " \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\" : 4 }, " + + " \"events\" : [ { " + + " \"segmentId\" : \"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 0, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " + + " \"robot\" : 1, " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"11._korpus_(NOVJ)\", " + + " \"language\" : \"sl\", " + + " \"newpage\" : \"0\", " + + " \"user\" : \"EmausBot\", " + + " \"count\" : 1.0, " + + " \"added\" : 39.0, " + + " \"delta\" : 39.0, " + + " \"variation\" : 39.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 1, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " + + " \"robot\" : 0, " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"112_U.S._580\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 70.0, " + + " \"delta\" : 70.0, " + + " \"variation\" : 70.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 2, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " + + " \"robot\" : 0, " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"113_U.S._243\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 77.0, " + + " \"delta\" : 77.0, " + + " \"variation\" : 77.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 3, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " + + " \"robot\" : 0, " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"113_U.S._73\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 70.0, " + + " \"delta\" : 70.0, " + + " \"variation\" : 70.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 4, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " + + " \"robot\" : 0, " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"113_U.S._756\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 68.0, " + + " \"delta\" : 68.0, " + + " \"variation\" : 68.0, " + + " \"deleted\" : 0.0 " + + " } " + + " } ] }} ]"; // Select query results as records (types defined by metastore) - private static final String SELECT_COLUMN_NAMES = "__time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted"; - private static final String SELECT_COLUMN_TYPES = "timestamp with local time zone,boolean,string,string,string,string,string,string,string,double,double,float,float,float"; - private static final Object[][] SELECT_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998400000L).atZone(ZoneOffset.UTC))), new BooleanWritable(true), - new Text("article"), new Text("0"), new Text("0"), - new Text("11._korpus_(NOVJ)"), new Text("sl"), new Text("0"), - new Text("EmausBot"), - new DoubleWritable(1.0d), new DoubleWritable(39.0d), new FloatWritable(39.0F), - new FloatWritable(39.0F), new FloatWritable(0.0F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998400000L).atZone(ZoneOffset.UTC))), new BooleanWritable(false), - new Text("article"), new Text("0"), new Text("0"), - new Text("112_U.S._580"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new DoubleWritable(1.0d), new DoubleWritable(70.0d), new FloatWritable(70.0F), - new FloatWritable(70.0F), new FloatWritable(0.0F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998412000L).atZone(ZoneOffset.UTC))), new BooleanWritable(false), - new Text("article"), new Text("0"), new Text("0"), - new Text("113_U.S._243"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new DoubleWritable(1.0d), new DoubleWritable(77.0d), new FloatWritable(77.0F), - new FloatWritable(77.0F), new FloatWritable(0.0F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998412000L).atZone(ZoneOffset.UTC))), new BooleanWritable(false), - new Text("article"), new Text("0"), new Text("0"), - new Text("113_U.S._73"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new DoubleWritable(1.0d), new DoubleWritable(70.0d), new FloatWritable(70.0F), - new FloatWritable(70.0F), new FloatWritable(0.0F) }, - new Object[] { new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998412000L).atZone(ZoneOffset.UTC))), new BooleanWritable(false), - new Text("article"), new Text("0"), new Text("0"), - new Text("113_U.S._756"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new DoubleWritable(1.0d), new DoubleWritable(68.0d), new FloatWritable(68.0F), - new FloatWritable(68.0F), new FloatWritable(0.0F) } - }; + private static final String + SELECT_COLUMN_NAMES = + "__time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted"; + private static final String + SELECT_COLUMN_TYPES = + "timestamp with local time zone,boolean,string,string,string,string,string,string,string,double,double,float,float,float"; + private static final Object[][] + SELECT_QUERY_RESULTS_RECORDS = + new Object[][] { new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998400000L).atZone(ZoneOffset.UTC))), + new BooleanWritable(true), new Text("article"), new Text("0"), new Text("0"), new Text("11._korpus_(NOVJ)"), + new Text("sl"), new Text("0"), new Text("EmausBot"), new DoubleWritable(1.0d), new DoubleWritable(39.0d), + new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(0.0F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998400000L).atZone(ZoneOffset.UTC))), + new BooleanWritable(false), new Text("article"), new Text("0"), new Text("0"), new Text("112_U.S._580"), + new Text("en"), new Text("1"), new Text("MZMcBride"), new DoubleWritable(1.0d), new DoubleWritable(70.0d), + new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998412000L).atZone(ZoneOffset.UTC))), + new BooleanWritable(false), new Text("article"), new Text("0"), new Text("0"), new Text("113_U.S._243"), + new Text("en"), new Text("1"), new Text("MZMcBride"), new DoubleWritable(1.0d), new DoubleWritable(77.0d), + new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(0.0F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998412000L).atZone(ZoneOffset.UTC))), + new BooleanWritable(false), new Text("article"), new Text("0"), new Text("0"), new Text("113_U.S._73"), + new Text("en"), new Text("1"), new Text("MZMcBride"), new DoubleWritable(1.0d), new DoubleWritable(70.0d), + new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) }, new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1356998412000L).atZone(ZoneOffset.UTC))), + new BooleanWritable(false), new Text("article"), new Text("0"), new Text("0"), new Text("113_U.S._756"), + new Text("en"), new Text("1"), new Text("MZMcBride"), new DoubleWritable(1.0d), new DoubleWritable(68.0d), + new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(0.0F) } }; // Scan query - private static final String SCAN_QUERY = - "{ \"queryType\": \"scan\", " - + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " - + " \"columns\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\",\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " - + " \"granularity\": \"all\", " - + " \"intervals\": [ \"2013-01-01/2013-01-02\" ]," - + " \"resultFormat\": \"compactedList\"," - + " \"limit\": 5" - + "}"; - - private static final String SCAN_QUERY_RESULTS = "[{" + private static final String + SCAN_QUERY = + "{ \"queryType\": \"scan\", " + + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " + + " \"columns\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\",\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " + + " \"granularity\": \"all\", " + + " \"intervals\": [ \"2013-01-01/2013-01-02\" ]," + + " \"resultFormat\": \"compactedList\"," + + " \"limit\": 5" + + "}"; + + private static final String + SCAN_QUERY_RESULTS = + "[{" + "\"segmentId\":\"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"," + "\"columns\":[\"__time\",\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\"," + "\"newpage\",\"user\",\"count\",\"added\",\"delta\",\"variation\",\"deleted\"]," @@ -612,118 +631,123 @@ + "[\"2013-01-01T00:00:12.000Z\", 0,\"article\",\"0\",\"0\",\"113_U.S._756\",\"en\",\"1\",\"MZMcBride\",1.0,68.0,68.0,68.0,0.0]" + "]}]"; + @Before public void setup() throws IOException { + tsQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + TIMESERIES_QUERY_RESULTS, + new TypeReference>>() { + })); - @Before - public void setup() throws IOException { - tsQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(TIMESERIES_QUERY_RESULTS, new TypeReference>>() { - })); - - topNQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER - .writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(TOPN_QUERY_RESULTS, new TypeReference>>() { + topNQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + TOPN_QUERY_RESULTS, + new TypeReference>>() { })); - groupByQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER - .writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(GROUP_BY_QUERY_RESULTS, - new TypeReference>() { - })); - groupByTimeExtractQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER - .writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(GB_TIME_EXTRACTIONS_RESULTS, new TypeReference>() { + groupByQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + GROUP_BY_QUERY_RESULTS, + new TypeReference>() { })); - groupByMonthExtractQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER - .writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(GB_MONTH_EXTRACTIONS_RESULTS, new TypeReference>() { + groupByTimeExtractQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + GB_TIME_EXTRACTIONS_RESULTS, + new TypeReference>() { })); - selectQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER - .writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(SELECT_QUERY_RESULTS, new TypeReference>>() { + groupByMonthExtractQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + GB_MONTH_EXTRACTIONS_RESULTS, + new TypeReference>() { + })); + selectQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + SELECT_QUERY_RESULTS, + new TypeReference>>() { })); - scanQueryResults = DruidStorageHandlerUtils.SMILE_MAPPER - .writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue(SCAN_QUERY_RESULTS, new TypeReference>() { + scanQueryResults = + DruidStorageHandlerUtils.SMILE_MAPPER.writeValueAsBytes(DruidStorageHandlerUtils.JSON_MAPPER.readValue( + SCAN_QUERY_RESULTS, + new TypeReference>() { })); } - /** - * Test the default behavior of the objects and object inspectors. - * @throws IOException - * @throws IllegalAccessException - * @throws IllegalArgumentException - * @throws SecurityException - * @throws NoSuchFieldException - * @throws JsonMappingException - * @throws JsonParseException - * @throws InvocationTargetException - * @throws NoSuchMethodException - */ - @Test - public void testDruidDeserializer() throws SerDeException, NoSuchFieldException, - SecurityException, IllegalArgumentException, IllegalAccessException, - IOException, InterruptedException, NoSuchMethodException, InvocationTargetException { + @Test public void testDruidDeserializer() + throws SerDeException, NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException, + IOException, InterruptedException, NoSuchMethodException, InvocationTargetException { // Create, initialize, and test the SerDe QTestDruidSerDe serDe = new QTestDruidSerDe(); Configuration conf = new Configuration(); Properties tbl; // Timeseries query - tbl = createPropertiesQuery("sample_datasource", Query.TIMESERIES, TIMESERIES_QUERY, - TIMESERIES_COLUMN_NAMES, TIMESERIES_COLUMN_TYPES - ); + tbl = + createPropertiesQuery("sample_datasource", + Query.TIMESERIES, + TIMESERIES_QUERY, + TIMESERIES_COLUMN_NAMES, + TIMESERIES_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.TIMESERIES, TIMESERIES_QUERY, tsQueryResults, - TIMESERIES_QUERY_RESULTS_RECORDS - ); + deserializeQueryResults(serDe, + Query.TIMESERIES, + TIMESERIES_QUERY, + tsQueryResults, + TIMESERIES_QUERY_RESULTS_RECORDS); // TopN query - tbl = createPropertiesQuery("sample_data", Query.TOPN, TOPN_QUERY, TOPN_COLUMN_NAMES, - TOPN_COLUMN_TYPES - ); + tbl = createPropertiesQuery("sample_data", Query.TOPN, TOPN_QUERY, TOPN_COLUMN_NAMES, TOPN_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.TOPN, TOPN_QUERY, topNQueryResults, - TOPN_QUERY_RESULTS_RECORDS - ); + deserializeQueryResults(serDe, Query.TOPN, TOPN_QUERY, topNQueryResults, TOPN_QUERY_RESULTS_RECORDS); // GroupBy query - tbl = createPropertiesQuery("sample_datasource", Query.GROUP_BY, GROUP_BY_QUERY, - GROUP_BY_COLUMN_NAMES, GROUP_BY_COLUMN_TYPES - ); + tbl = + createPropertiesQuery("sample_datasource", + Query.GROUP_BY, + GROUP_BY_QUERY, + GROUP_BY_COLUMN_NAMES, + GROUP_BY_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.GROUP_BY, GROUP_BY_QUERY, groupByQueryResults, - GROUP_BY_QUERY_RESULTS_RECORDS - ); - - tbl = createPropertiesQuery("sample_datasource", Query.GROUP_BY, GB_TIME_EXTRACTIONS, - GB_TIME_EXTRACTIONS_COLUMN_NAMES, GB_TIME_EXTRACTIONS_COLUMN_TYPES - ); + deserializeQueryResults(serDe, Query.GROUP_BY, GROUP_BY_QUERY, groupByQueryResults, GROUP_BY_QUERY_RESULTS_RECORDS); + + tbl = + createPropertiesQuery("sample_datasource", + Query.GROUP_BY, + GB_TIME_EXTRACTIONS, + GB_TIME_EXTRACTIONS_COLUMN_NAMES, + GB_TIME_EXTRACTIONS_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.GROUP_BY, GB_TIME_EXTRACTIONS, - groupByTimeExtractQueryResults, GROUP_BY_QUERY_EXTRACTION_RESULTS_RECORDS - ); - - tbl = createPropertiesQuery("sample_datasource", Query.GROUP_BY, GB_MONTH_EXTRACTIONS, - GB_MONTH_EXTRACTIONS_COLUMN_NAMES, GB_MONTH_EXTRACTIONS_COLUMN_TYPES - ); + deserializeQueryResults(serDe, + Query.GROUP_BY, + GB_TIME_EXTRACTIONS, + groupByTimeExtractQueryResults, + GROUP_BY_QUERY_EXTRACTION_RESULTS_RECORDS); + + tbl = + createPropertiesQuery("sample_datasource", + Query.GROUP_BY, + GB_MONTH_EXTRACTIONS, + GB_MONTH_EXTRACTIONS_COLUMN_NAMES, + GB_MONTH_EXTRACTIONS_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.GROUP_BY, GB_MONTH_EXTRACTIONS, - groupByMonthExtractQueryResults, GB_MONTH_EXTRACTION_RESULTS_RECORDS - ); + deserializeQueryResults(serDe, + Query.GROUP_BY, + GB_MONTH_EXTRACTIONS, + groupByMonthExtractQueryResults, + GB_MONTH_EXTRACTION_RESULTS_RECORDS); // Select query - tbl = createPropertiesQuery("wikipedia", Query.SELECT, SELECT_QUERY, SELECT_COLUMN_NAMES, - SELECT_COLUMN_TYPES - ); + tbl = createPropertiesQuery("wikipedia", Query.SELECT, SELECT_QUERY, SELECT_COLUMN_NAMES, SELECT_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.SELECT, SELECT_QUERY, selectQueryResults, - SELECT_QUERY_RESULTS_RECORDS - ); + deserializeQueryResults(serDe, Query.SELECT, SELECT_QUERY, selectQueryResults, SELECT_QUERY_RESULTS_RECORDS); // Scan query -- results should be same as select query - tbl = createPropertiesQuery("wikipedia", Query.SCAN, SCAN_QUERY, SELECT_COLUMN_NAMES, - SELECT_COLUMN_TYPES - ); + tbl = createPropertiesQuery("wikipedia", Query.SCAN, SCAN_QUERY, SELECT_COLUMN_NAMES, SELECT_COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeQueryResults(serDe, Query.SCAN, SCAN_QUERY, scanQueryResults, - SELECT_QUERY_RESULTS_RECORDS - ); + deserializeQueryResults(serDe, Query.SCAN, SCAN_QUERY, scanQueryResults, SELECT_QUERY_RESULTS_RECORDS); } - private static Properties createPropertiesQuery(String dataSource, String queryType, - String jsonQuery, String columnNames, String columnTypes - ) { + private static Properties createPropertiesQuery(String dataSource, + String queryType, + String jsonQuery, + String columnNames, + String columnTypes) { Properties tbl = new Properties(); // Set the configuration parameters @@ -735,28 +759,29 @@ private static Properties createPropertiesQuery(String dataSource, String queryT return tbl; } - @SuppressWarnings("unchecked") - private void deserializeQueryResults(DruidSerDe serDe, String queryType, String jsonQuery, - byte[] resultString, Object[][] records - ) throws SerDeException, IOException, NoSuchFieldException, SecurityException, - IllegalArgumentException, IllegalAccessException, InterruptedException, - NoSuchMethodException, InvocationTargetException { + @SuppressWarnings("unchecked") private void deserializeQueryResults(DruidSerDe serDe, + String queryType, + String jsonQuery, + byte[] resultString, + Object[][] records) + throws SerDeException, IOException, NoSuchFieldException, SecurityException, IllegalArgumentException, + IllegalAccessException, InterruptedException, NoSuchMethodException, InvocationTargetException { // Initialize HttpClient httpClient = mock(HttpClient.class); SettableFuture futureResult = SettableFuture.create(); futureResult.set(new ByteArrayInputStream(resultString)); when(httpClient.go(anyObject(), any(HttpResponseHandler.class))).thenReturn(futureResult); - DruidQueryRecordReader reader = DruidQueryBasedInputFormat.getDruidQueryReader(queryType); + DruidQueryRecordReader reader = DruidQueryBasedInputFormat.getDruidQueryReader(queryType); - final HiveDruidSplit split = new HiveDruidSplit(jsonQuery, - new Path("empty"), - new String[] { "testing_host" } - ); + final HiveDruidSplit split = new HiveDruidSplit(jsonQuery, new Path("empty"), new String[] { "testing_host" }); - reader.initialize(split, new Configuration(), DruidStorageHandlerUtils.JSON_MAPPER, - DruidStorageHandlerUtils.SMILE_MAPPER, httpClient - ); + assert reader != null; + reader.initialize(split, + new Configuration(), + DruidStorageHandlerUtils.JSON_MAPPER, + DruidStorageHandlerUtils.SMILE_MAPPER, + httpClient); StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector(); List fieldRefs = oi.getAllStructFieldRefs(); @@ -768,9 +793,7 @@ private void deserializeQueryResults(DruidSerDe serDe, String queryType, String Object[] expectedFieldsData = records[pos]; assertEquals(expectedFieldsData.length, fieldRefs.size()); for (int i = 0; i < fieldRefs.size(); i++) { - assertEquals("Field " + i + " type", expectedFieldsData[i].getClass(), - row.get(i).getClass() - ); + assertEquals("Field " + i + " type", expectedFieldsData[i].getClass(), row.get(i).getClass()); Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i)); assertEquals("Field " + i, expectedFieldsData[i], fieldData); } @@ -778,17 +801,19 @@ private void deserializeQueryResults(DruidSerDe serDe, String queryType, String } assertEquals(pos, records.length); - // Check mapreduce path futureResult = SettableFuture.create(); futureResult.set(new ByteArrayInputStream(resultString)); when(httpClient.go(anyObject(), any(HttpResponseHandler.class))).thenReturn(futureResult); reader = DruidQueryBasedInputFormat.getDruidQueryReader(queryType); - reader.initialize(split, new Configuration(), DruidStorageHandlerUtils.JSON_MAPPER, - DruidStorageHandlerUtils.SMILE_MAPPER, httpClient - ); - - pos = 0; + assert reader != null; + reader.initialize(split, + new Configuration(), + DruidStorageHandlerUtils.JSON_MAPPER, + DruidStorageHandlerUtils.SMILE_MAPPER, + httpClient); + + pos = 0; while (reader.nextKeyValue()) { List row = (List) serDe.deserialize(reader.getCurrentValue()); Object[] expectedFieldsData = records[pos]; @@ -803,25 +828,23 @@ private void deserializeQueryResults(DruidSerDe serDe, String queryType, String assertEquals(pos, records.length); } - private static final String COLUMN_NAMES = "__time,c0,c1,c2,c3,c4,c5,c6,c7,c8"; - private static final String COLUMN_TYPES = "timestamp with local time zone,string,char(6),varchar(8),double,float,bigint,int,smallint,tinyint"; - private static final Object[] ROW_OBJECT = new Object[] { - new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("dim1_val"), - new HiveCharWritable(new HiveChar("dim2_v", 6)), - new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), - new DoubleWritable(10669.3D), - new FloatWritable(10669.45F), - new LongWritable(1113939), - new IntWritable(1112123), - new ShortWritable((short) 12), - new ByteWritable((byte) 0), - new TimestampWritableV2(Timestamp.ofEpochSecond(1377907200L)) // granularity - }; - private static final DruidWritable DRUID_WRITABLE = new DruidWritable( - ImmutableMap.builder() - .put("__time", 1377907200000L) + private static final String + COLUMN_TYPES = + "timestamp with local time zone,string,char(6),varchar(8),double,float,bigint,int,smallint,tinyint"; + private static final Object[] + ROW_OBJECT = + new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), + new Text("dim1_val"), new HiveCharWritable(new HiveChar("dim2_v", 6)), + new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), new DoubleWritable(10669.3D), + new FloatWritable(10669.45F), new LongWritable(1113939), new IntWritable(1112123), + new ShortWritable((short) 12), new ByteWritable((byte) 0), + new TimestampWritableV2(Timestamp.ofEpochSecond(1377907200L)) // granularity + }; + private static final DruidWritable + DRUID_WRITABLE = + new DruidWritable(ImmutableMap.builder().put("__time", 1377907200000L) .put("c0", "dim1_val") .put("c1", "dim2_v") .put("c2", "dim3_val") @@ -834,24 +857,10 @@ private void deserializeQueryResults(DruidSerDe serDe, String queryType, String .put("__time_granularity", 1377907200000L) .build()); - /** - * Test the default behavior of the objects and object inspectors. - * @throws IOException - * @throws IllegalAccessException - * @throws IllegalArgumentException - * @throws SecurityException - * @throws NoSuchFieldException - * @throws JsonMappingException - * @throws JsonParseException - * @throws InvocationTargetException - * @throws NoSuchMethodException - */ - @Test - public void testDruidObjectSerializer() - throws SerDeException, JsonParseException, JsonMappingException, - NoSuchFieldException, SecurityException, IllegalArgumentException, - IllegalAccessException, IOException, InterruptedException, - NoSuchMethodException, InvocationTargetException { + @Test public void testDruidObjectSerializer() + throws SerDeException, NoSuchFieldException, SecurityException, + IllegalArgumentException, IllegalAccessException, IOException, InterruptedException, NoSuchMethodException, + InvocationTargetException { // Create, initialize, and test the SerDe DruidSerDe serDe = new DruidSerDe(); Configuration conf = new Configuration(); @@ -862,12 +871,9 @@ public void testDruidObjectSerializer() serializeObject(tbl, serDe, ROW_OBJECT, DRUID_WRITABLE); } - @Rule - public ExpectedException expectedEx = ExpectedException.none(); + @Rule public ExpectedException expectedEx = ExpectedException.none(); - @Test - public void testDruidObjectSerializerwithNullTimestamp() - throws Exception { + @Test public void testDruidObjectSerializerwithNullTimestamp() throws Exception { // Create, initialize, and test the SerDe DruidSerDe serDe = new DruidSerDe(); Configuration conf = new Configuration(); @@ -875,19 +881,13 @@ public void testDruidObjectSerializerwithNullTimestamp() // Mixed source (all types) tbl = createPropertiesSource(COLUMN_NAMES, COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - Object[] row = new Object[] { - null, - new Text("dim1_val"), - new HiveCharWritable(new HiveChar("dim2_v", 6)), - new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), - new DoubleWritable(10669.3D), - new FloatWritable(10669.45F), - new LongWritable(1113939), - new IntWritable(1112123), - new ShortWritable((short) 12), - new ByteWritable((byte) 0), - null // granularity - }; + Object[] + row = + new Object[] { null, new Text("dim1_val"), new HiveCharWritable(new HiveChar("dim2_v", 6)), + new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), new DoubleWritable(10669.3D), + new FloatWritable(10669.45F), new LongWritable(1113939), new IntWritable(1112123), + new ShortWritable((short) 12), new ByteWritable((byte) 0), null // granularity + }; expectedEx.expect(NullPointerException.class); expectedEx.expectMessage("Timestamp column cannot have null value"); // should fail as timestamp is null @@ -903,58 +903,45 @@ private static Properties createPropertiesSource(String columnNames, String colu return tbl; } - private static void serializeObject(Properties properties, DruidSerDe serDe, - Object[] rowObject, DruidWritable druidWritable) throws SerDeException { + private static void serializeObject(Properties properties, + DruidSerDe serDe, + Object[] rowObject, + DruidWritable druidWritable) throws SerDeException { // Build OI with timestamp granularity column - final List columnNames = new ArrayList<>(); - final List columnTypes = new ArrayList<>(); - List inspectors = new ArrayList<>(); - columnNames.addAll(Utilities.getColumnNames(properties)); + final List columnNames = new ArrayList<>(Utilities.getColumnNames(properties)); columnNames.add(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME); - columnTypes.addAll(Lists.transform(Utilities.getColumnTypes(properties), - new Function() { - @Override - public PrimitiveTypeInfo apply(String type) { - return TypeInfoFactory.getPrimitiveTypeInfo(type); - } - } - )); + final List + columnTypes = + Utilities.getColumnTypes(properties) + .stream() + .map(TypeInfoFactory::getPrimitiveTypeInfo) + .collect(Collectors.toList()); columnTypes.add(TypeInfoFactory.getPrimitiveTypeInfo("timestamp")); - inspectors.addAll(Lists.transform(columnTypes, - new Function() { - @Override - public ObjectInspector apply(PrimitiveTypeInfo type) { - return PrimitiveObjectInspectorFactory - .getPrimitiveWritableObjectInspector(type); - } - } - )); - ObjectInspector inspector = ObjectInspectorFactory - .getStandardStructObjectInspector(columnNames, inspectors); + List + inspectors = + columnTypes.stream().map(PrimitiveObjectInspectorFactory::getPrimitiveWritableObjectInspector).collect( + Collectors.toList()); + ObjectInspector inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); // Serialize DruidWritable writable = (DruidWritable) serDe.serialize(rowObject, inspector); // Check result assertEquals(druidWritable.getValue().size(), writable.getValue().size()); - for (Entry e: druidWritable.getValue().entrySet()) { + for (Entry e : druidWritable.getValue().entrySet()) { assertEquals(e.getValue(), writable.getValue().get(e.getKey())); } } - private static final Object[] ROW_OBJECT_2 = new Object[] { - new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), - new Text("dim1_val"), - new HiveCharWritable(new HiveChar("dim2_v", 6)), - new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), - new DoubleWritable(10669.3D), - new FloatWritable(10669.45F), - new LongWritable(1113939), - new IntWritable(1112123), - new ShortWritable((short) 12), - new ByteWritable((byte) 0) - }; - private static final DruidWritable DRUID_WRITABLE_2 = new DruidWritable( - ImmutableMap.builder() - .put("__time", 1377907200000L) + private static final Object[] + ROW_OBJECT_2 = + new Object[] { + new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1377907200000L).atZone(ZoneOffset.UTC))), + new Text("dim1_val"), new HiveCharWritable(new HiveChar("dim2_v", 6)), + new HiveVarcharWritable(new HiveVarchar("dim3_val", 8)), new DoubleWritable(10669.3D), + new FloatWritable(10669.45F), new LongWritable(1113939), new IntWritable(1112123), + new ShortWritable((short) 12), new ByteWritable((byte) 0) }; + private static final DruidWritable + DRUID_WRITABLE_2 = + new DruidWritable(ImmutableMap.builder().put("__time", 1377907200000L) .put("c0", "dim1_val") .put("c1", "dim2_v") .put("c2", "dim3_val") @@ -966,12 +953,9 @@ public ObjectInspector apply(PrimitiveTypeInfo type) { .put("c8", (byte) 0) .build()); - @Test - public void testDruidObjectDeserializer() - throws SerDeException, JsonParseException, JsonMappingException, - NoSuchFieldException, SecurityException, IllegalArgumentException, - IllegalAccessException, IOException, InterruptedException, - NoSuchMethodException, InvocationTargetException { + @Test public void testDruidObjectDeserializer() + throws SerDeException, NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException, + IOException, InterruptedException, NoSuchMethodException, InvocationTargetException { // Create, initialize, and test the SerDe DruidSerDe serDe = new DruidSerDe(); Configuration conf = new Configuration(); @@ -979,12 +963,12 @@ public void testDruidObjectDeserializer() // Mixed source (all types) tbl = createPropertiesSource(COLUMN_NAMES, COLUMN_TYPES); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); - deserializeObject(tbl, serDe, ROW_OBJECT_2, DRUID_WRITABLE_2); + deserializeObject(serDe, ROW_OBJECT_2, DRUID_WRITABLE_2); } - @SuppressWarnings("unchecked") - private static void deserializeObject(Properties properties, DruidSerDe serDe, - Object[] rowObject, DruidWritable druidWritable) throws SerDeException { + @SuppressWarnings("unchecked") private static void deserializeObject(DruidSerDe serDe, + Object[] rowObject, + DruidWritable druidWritable) throws SerDeException { // Deserialize List object = (List) serDe.deserialize(druidWritable); // Check result diff --git druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java index cb8fa3919b..c6f6e4659e 100644 --- druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java +++ druid-handler/src/test/org/apache/hadoop/hive/ql/io/TestDruidRecordWriter.java @@ -18,14 +18,13 @@ package org.apache.hadoop.hive.ql.io; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; 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; @@ -69,38 +68,36 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; -public class TestDruidRecordWriter { - private ObjectMapper objectMapper = DruidStorageHandlerUtils.JSON_MAPPER; +@SuppressWarnings("ConstantConditions") public class TestDruidRecordWriter { + private final ObjectMapper objectMapper = DruidStorageHandlerUtils.JSON_MAPPER; private static final Interval INTERVAL_FULL = new Interval("2014-10-22T00:00:00Z/P1D"); @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private DruidRecordWriter druidRecordWriter; - final List> expectedRows = ImmutableList.of( - ImmutableMap.of( + ImmutableMap.of( DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, DateTime.parse("2014-10-22T00:00:00.000Z").getMillis(), "host", ImmutableList.of("a.example.com"), "visited_sum", 190L, "unique_hosts", 1.0d ), - ImmutableMap.of( + ImmutableMap.of( DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, DateTime.parse("2014-10-22T01:00:00.000Z").getMillis(), "host", ImmutableList.of("b.example.com"), "visited_sum", 175L, "unique_hosts", 1.0d ), - ImmutableMap.of( + ImmutableMap.of( DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, DateTime.parse("2014-10-22T02:00:00.000Z").getMillis(), "host", ImmutableList.of("c.example.com"), @@ -128,11 +125,11 @@ public void testWrite() throws IOException, SegmentLoadingException { final InputRowParser inputRowParser = new MapInputRowParser(new TimeAndDimsParseSpec( new TimestampSpec(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN, "auto", null), - new DimensionsSpec(ImmutableList.of(new StringDimensionSchema("host")), + new DimensionsSpec(ImmutableList.of(new StringDimensionSchema("host")), null, null ) )); - final Map parserMap = objectMapper.convertValue(inputRowParser, Map.class); + final Map parserMap = objectMapper.convertValue(inputRowParser, new TypeReference>(){}); DataSchema dataSchema = new DataSchema( dataSourceName, @@ -163,26 +160,18 @@ public void testWrite() throws IOException, SegmentLoadingException { Path segmentDescriptroPath = new Path(workingDir.getAbsolutePath(), DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME ); - druidRecordWriter = new DruidRecordWriter(dataSchema, tuningConfig, dataSegmentPusher, 20, - segmentDescriptroPath, localFileSystem - ); - - List druidWritables = Lists.transform(expectedRows, - new Function, DruidWritable>() { - @Nullable - @Override - public DruidWritable apply(@Nullable ImmutableMap input - ) { - return new DruidWritable(ImmutableMap.builder().putAll(input) - .put(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME, - Granularities.DAY.bucketStart( - new DateTime((long) input - .get(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN))) - .getMillis() - ).build()); - } - } - ); + DruidRecordWriter + druidRecordWriter = + new DruidRecordWriter(dataSchema, tuningConfig, dataSegmentPusher, 20, segmentDescriptroPath, localFileSystem); + + List druidWritables = + expectedRows.stream() + .map(input -> new DruidWritable(ImmutableMap.builder().putAll(input) + .put(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME, + Granularities.DAY.bucketStart(new DateTime((long) input.get(DruidStorageHandlerUtils.DEFAULT_TIMESTAMP_COLUMN))) + .getMillis()) + .build())) + .collect(Collectors.toList()); for (DruidWritable druidWritable : druidWritables) { druidRecordWriter.write(druidWritable); } @@ -243,9 +232,9 @@ private void verifyRows(List> expectedRows, @Test public void testSerDesr() throws IOException { String segment = "{\"dataSource\":\"datasource2015\",\"interval\":\"2015-06-01T00:00:00.000-04:00/2015-06-02T00:00:00.000-04:00\",\"version\":\"2016-11-04T19:24:01.732-04:00\",\"loadSpec\":{\"type\":\"hdfs\",\"path\":\"hdfs://cn105-10.l42scl.hortonworks.com:8020/apps/hive/warehouse/druid.db/.hive-staging_hive_2016-11-04_19-23-50_168_1550339856804207572-1/_task_tmp.-ext-10002/_tmp.000000_0/datasource2015/20150601T000000.000-0400_20150602T000000.000-0400/2016-11-04T19_24_01.732-04_00/0/index.zip\"},\"dimensions\":\"dimension1\",\"metrics\":\"bigint\",\"shardSpec\":{\"type\":\"linear\",\"partitionNum\":0},\"binaryVersion\":9,\"size\":1765,\"identifier\":\"datasource2015_2015-06-01T00:00:00.000-04:00_2015-06-02T00:00:00.000-04:00_2016-11-04T19:24:01.732-04:00\"}"; - DataSegment dataSegment = objectMapper.reader(DataSegment.class) + DataSegment dataSegment = objectMapper.readerFor(DataSegment.class) .readValue(segment); - Assert.assertTrue(dataSegment.getDataSource().equals("datasource2015")); + Assert.assertEquals("datasource2015", dataSegment.getDataSource()); } } diff --git ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index 98fae6cd2f..ab348c977b 100644 --- ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -186,16 +186,6 @@ POSTHOOK: query: Select `__partition`, `__offset`, `user` from kafka_table wher POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -0 0 nuclear -0 1 speed -0 2 masterYi -0 3 triplets -0 4 stringer -0 5 nuclear -0 6 speed -0 7 masterYi -0 8 triplets -0 9 stringer PREHOOK: query: Select count(*) from kafka_table where `__partition` = 1 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table