diff --git checkstyle/checkstyle.xml checkstyle/checkstyle.xml index 12e166311b..756b997cff 100644 --- checkstyle/checkstyle.xml +++ checkstyle/checkstyle.xml @@ -163,8 +163,12 @@ + + + + + - diff --git data/files/datasets/druid_table_alltypesorc/load.hive.sql data/files/datasets/druid_table_alltypesorc/load.hive.sql index 5fde266a01..189c0aed25 100644 --- data/files/datasets/druid_table_alltypesorc/load.hive.sql +++ data/files/datasets/druid_table_alltypesorc/load.hive.sql @@ -18,7 +18,7 @@ OVERWRITE INTO TABLE alltypesorc1; CREATE EXTERNAL TABLE druid_table_alltypesorc STORED BY 'org.apache.hadoop.hive.druid.DruidStorageHandler' -TBLPROPERTIES ("druid.segment.granularity" = "HOUR", "druid.query.granularity" = "MINUTE") +TBLPROPERTIES ("druid.segment.granularity" = "HOUR", "druid.query.granularity" = "SECOND") AS SELECT cast (`ctimestamp1` as timestamp with local time zone) as `__time`, cstring1, @@ -32,5 +32,3 @@ SELECT cast (`ctimestamp1` as timestamp with local time zone) as `__time`, cboolean1, cboolean2 FROM alltypesorc1 where ctimestamp1 IS NOT NULL; - -DROP TABLE alltypesorc1; diff --git data/scripts/q_test_cleanup.sql data/scripts/q_test_cleanup.sql index 1c59381aa0..731062efec 100644 --- data/scripts/q_test_cleanup.sql +++ data/scripts/q_test_cleanup.sql @@ -24,3 +24,4 @@ DROP TABLE IF EXISTS cbo_t3; DROP TABLE IF EXISTS src_cbo; DROP TABLE IF EXISTS part; DROP TABLE IF EXISTS lineitem; +DROP TABLE IF EXISTS alltypesorc1; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java index e0e29a3c6d..c5dc1e8eba 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidKafkaUtils.java @@ -134,9 +134,7 @@ static KafkaSupervisorSpec createKafkaSupervisorSpec(Table table, builder.put(KafkaSupervisorIOConfig.BOOTSTRAP_SERVERS_KEY, kafkaServers); for (Map.Entry entry : table.getParameters().entrySet()) { if (entry.getKey().startsWith(DruidConstants.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX)) { - String - propertyName = - entry.getKey().substring(DruidConstants.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX.length()); + String propertyName = entry.getKey().substring(DruidConstants.DRUID_KAFKA_CONSUMER_PROPERTY_PREFIX.length()); builder.put(propertyName, entry.getValue()); } } @@ -178,61 +176,47 @@ static boolean isKafkaStreamingTable(Table table) { return DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.KAFKA_TOPIC) != null; } - static InputRowParser getInputRowParser(Table table, - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec - ) { + static InputRowParser getInputRowParser(Table table, TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) { String parseSpecFormat = DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_FORMAT); // Default case JSON - if(parseSpecFormat == null || parseSpecFormat.equalsIgnoreCase("json")) { - return new StringInputRowParser( - new JSONParseSpec(timestampSpec, - dimensionsSpec, - null, - null - ), "UTF-8"); - } else if(parseSpecFormat.equalsIgnoreCase("csv")){ - return new StringInputRowParser( - new CSVParseSpec( - timestampSpec, - dimensionsSpec, - DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_LIST_DELIMITER), - DruidStorageHandlerUtils.getListProperty(table, DruidConstants.DRUID_PARSE_SPEC_COLUMNS), - DruidStorageHandlerUtils.getBooleanProperty(table, DruidConstants.DRUID_PARSE_SPEC_HAS_HEADER_ROWS, false), - DruidStorageHandlerUtils.getIntegerProperty(table, DruidConstants.DRUID_PARSE_SPEC_SKIP_HEADER_ROWS, 0) - ), "UTF-8"); - } else if (parseSpecFormat.equalsIgnoreCase("delimited")){ - return new StringInputRowParser( - new DelimitedParseSpec( - timestampSpec, - dimensionsSpec, - DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_DELIMITER), - DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_LIST_DELIMITER), - DruidStorageHandlerUtils.getListProperty(table, DruidConstants.DRUID_PARSE_SPEC_COLUMNS), - DruidStorageHandlerUtils.getBooleanProperty(table, DruidConstants.DRUID_PARSE_SPEC_HAS_HEADER_ROWS, false), - DruidStorageHandlerUtils.getIntegerProperty(table, DruidConstants.DRUID_PARSE_SPEC_SKIP_HEADER_ROWS, 0) - ), "UTF-8"); - } else if(parseSpecFormat.equalsIgnoreCase("avro")) { + if ((parseSpecFormat == null) || "json".equalsIgnoreCase(parseSpecFormat)) { + return new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, null, null), "UTF-8"); + } else if ("csv".equalsIgnoreCase(parseSpecFormat)) { + return new StringInputRowParser(new CSVParseSpec(timestampSpec, + dimensionsSpec, + DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_LIST_DELIMITER), + DruidStorageHandlerUtils.getListProperty(table, DruidConstants.DRUID_PARSE_SPEC_COLUMNS), + DruidStorageHandlerUtils.getBooleanProperty(table, DruidConstants.DRUID_PARSE_SPEC_HAS_HEADER_ROWS, false), + DruidStorageHandlerUtils.getIntegerProperty(table, DruidConstants.DRUID_PARSE_SPEC_SKIP_HEADER_ROWS, 0)), + "UTF-8"); + } else if ("delimited".equalsIgnoreCase(parseSpecFormat)) { + return new StringInputRowParser(new DelimitedParseSpec(timestampSpec, + dimensionsSpec, + DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_DELIMITER), + DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.DRUID_PARSE_SPEC_LIST_DELIMITER), + DruidStorageHandlerUtils.getListProperty(table, DruidConstants.DRUID_PARSE_SPEC_COLUMNS), + DruidStorageHandlerUtils.getBooleanProperty(table, DruidConstants.DRUID_PARSE_SPEC_HAS_HEADER_ROWS, false), + DruidStorageHandlerUtils.getIntegerProperty(table, DruidConstants.DRUID_PARSE_SPEC_SKIP_HEADER_ROWS, 0)), + "UTF-8"); + } else if ("avro".equalsIgnoreCase(parseSpecFormat)) { try { String avroSchemaLiteral = DruidStorageHandlerUtils.getTableProperty(table, DruidConstants.AVRO_SCHEMA_LITERAL); - Preconditions.checkNotNull(avroSchemaLiteral, - "Please specify avro schema literal when using avro parser" - ); - Map avroSchema = JSON_MAPPER - .readValue(avroSchemaLiteral, new TypeReference>() { - }); - return new AvroStreamInputRowParser(new AvroParseSpec( - timestampSpec, - dimensionsSpec, - null - ), new InlineSchemaAvroBytesDecoder(avroSchema)); + Preconditions.checkNotNull(avroSchemaLiteral, "Please specify avro schema literal when using avro parser"); + Map + avroSchema = + JSON_MAPPER.readValue(avroSchemaLiteral, new TypeReference>() { + }); + return new AvroStreamInputRowParser(new AvroParseSpec(timestampSpec, dimensionsSpec, null), + new InlineSchemaAvroBytesDecoder(avroSchema)); } catch (Exception e) { throw new IllegalStateException("Exception while creating avro schema", e); } } - throw new IllegalArgumentException("Invalid parse spec format [" + parseSpecFormat+"]. " - + "Supported types are : json, csv, tsv, avro"); + throw new IllegalArgumentException("Invalid parse spec format [" + + parseSpecFormat + + "]. " + + "Supported types are : json, csv, tsv, avro"); } } 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 7434559532..94a3a27ad4 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java @@ -672,7 +672,7 @@ private static boolean safeNonRecursiveDelete(FileSystem fs, Path path) { 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 + // 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 && MetaStoreUtils.isExternalTablePurge(table)) { @@ -954,7 +954,7 @@ private int getMaxRetryCount() { } return new DruidStorageHandlerInfo(kafkaSupervisorReport); } else { - // TODO: Currently we do not expose any runtime info for non-streaming tables. + // 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/DruidStorageHandlerUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java index 6dc97d53b7..27c4577c9e 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java @@ -149,8 +149,7 @@ * Utils class for Druid storage handler. */ public final class DruidStorageHandlerUtils { - private DruidStorageHandlerUtils () { - + private DruidStorageHandlerUtils() { } private static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandlerUtils.class); @@ -462,7 +461,7 @@ static boolean disableDataSource(SQLMetadataConnector connector, 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. + // case metadata storage has segments 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. " @@ -672,7 +671,7 @@ static int getIntegerProperty(Table table, String propertyName, int defaultVal) } @Nullable public static List getListProperty(Table table, String propertyName) { - List rv = new ArrayList(); + List rv = new ArrayList<>(); String values = getTableProperty(table, propertyName); if(values == null) { return null; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/conf/DruidConstants.java druid-handler/src/java/org/apache/hadoop/hive/druid/conf/DruidConstants.java index 242f7be4dd..797d597500 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/conf/DruidConstants.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/conf/DruidConstants.java @@ -17,7 +17,12 @@ */ package org.apache.hadoop.hive.druid.conf; -public class DruidConstants { +/** + * Utility class for Druid Constants. + */ +public final class DruidConstants { + private DruidConstants() { + } public static final String DRUID_QUERY_FETCH = "druid.query.fetch"; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/conf/package-info.java druid-handler/src/java/org/apache/hadoop/hive/druid/conf/package-info.java new file mode 100644 index 0000000000..1049acb2a9 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/conf/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.druid.conf; 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 c1e0e75f98..7137fd2815 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 @@ -45,6 +45,8 @@ import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader; import org.apache.hadoop.hive.druid.serde.DruidWritable; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedSupport; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapred.JobConf; @@ -60,7 +62,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; import java.io.IOException; import java.io.InputStream; import java.net.URL; @@ -75,11 +76,11 @@ * and parse the results. */ public class DruidQueryBasedInputFormat extends InputFormat - implements org.apache.hadoop.mapred.InputFormat { + implements org.apache.hadoop.mapred.InputFormat, VectorizedInputFormatInterface { protected static final Logger LOG = LoggerFactory.getLogger(DruidQueryBasedInputFormat.class); - @Nullable public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) { + public static DruidQueryRecordReader getDruidQueryReader(String druidQueryType) { switch (druidQueryType) { case Query.TIMESERIES: return new DruidTimeseriesQueryRecordReader(); @@ -92,26 +93,20 @@ case Query.SCAN: return new DruidScanQueryRecordReader(); default: - return null; + throw new IllegalStateException("Druid query type " + druidQueryType + " not recognized"); } } - @Override - public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf job, int numSplits) - throws IOException { + @Override public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { return getInputSplits(job); } - @Override - public List getSplits(JobContext context) throws IOException, InterruptedException { + @Override public List getSplits(JobContext context) throws IOException, InterruptedException { return Arrays.asList(getInputSplits(context.getConfiguration())); } - @SuppressWarnings("deprecation") - protected HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException { - String address = HiveConf.getVar(conf, - HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS - ); + @SuppressWarnings("deprecation") protected HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException { + String address = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS); String queryId = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID); if (StringUtils.isEmpty(address)) { throw new IOException("Druid broker address not specified in configuration"); @@ -193,16 +188,18 @@ hosts[j] = locatedSD.getLocations().get(j).getHost(); } // Create partial Select query - final SegmentDescriptor newSD = new SegmentDescriptor( - locatedSD.getInterval(), locatedSD.getVersion(), locatedSD.getPartitionNumber()); + final SegmentDescriptor + newSD = + new SegmentDescriptor(locatedSD.getInterval(), locatedSD.getVersion(), locatedSD.getPartitionNumber()); //@TODO This is fetching all the rows at once from broker or multiple historical nodes // Move to use scan query to avoid GC back pressure on the nodes // https://issues.apache.org/jira/browse/HIVE-17627 - final SelectQuery partialQuery = query - .withQuerySegmentSpec(new MultipleSpecificSegmentSpec(Lists.newArrayList(newSD))) + final SelectQuery + partialQuery = + query.withQuerySegmentSpec(new MultipleSpecificSegmentSpec(Lists.newArrayList(newSD))) .withPagingSpec(PagingSpec.newSpec(Integer.MAX_VALUE)); - splits[i] = new HiveDruidSplit(DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), - dummyPath, hosts); + splits[i] = + new HiveDruidSplit(DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath, hosts); } return splits; } @@ -221,8 +218,7 @@ }; } - final List segmentDescriptors = fetchLocatedSegmentDescriptors( - address, query); + final List segmentDescriptors = fetchLocatedSegmentDescriptors(address, query); // Create one input split for each segment final int numSplits = segmentDescriptors.size(); @@ -237,28 +233,31 @@ hosts[locatedSD.getLocations().size()] = address; // Create partial Select query - final SegmentDescriptor newSD = new SegmentDescriptor( - locatedSD.getInterval(), locatedSD.getVersion(), locatedSD.getPartitionNumber()); - final Query partialQuery = query - .withQuerySegmentSpec(new MultipleSpecificSegmentSpec(Lists.newArrayList(newSD))); - splits[i] = new HiveDruidSplit(DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), - dummyPath, hosts); + final SegmentDescriptor + newSD = + new SegmentDescriptor(locatedSD.getInterval(), locatedSD.getVersion(), locatedSD.getPartitionNumber()); + final Query partialQuery = query.withQuerySegmentSpec(new MultipleSpecificSegmentSpec(Lists.newArrayList(newSD))); + splits[i] = + new HiveDruidSplit(DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath, hosts); } return splits; } - private static List fetchLocatedSegmentDescriptors(String address, - BaseQuery query) throws IOException { - final String intervals = - StringUtils.join(query.getIntervals(), ","); // Comma-separated intervals without brackets - final String request = String.format( - "http://%s/druid/v2/datasources/%s/candidates?intervals=%s", - address, query.getDataSource().getNames().get(0), URLEncoder.encode(intervals, "UTF-8")); + private static List fetchLocatedSegmentDescriptors(String address, BaseQuery query) + throws IOException { + final String intervals = StringUtils.join(query.getIntervals(), ","); // Comma-separated intervals without brackets + final String + request = + String.format("http://%s/druid/v2/datasources/%s/candidates?intervals=%s", + address, + query.getDataSource().getNames().get(0), + URLEncoder.encode(intervals, "UTF-8")); LOG.debug("sending request {} to query for segments", request); final InputStream response; try { - response = DruidStorageHandlerUtils - .submitRequest(DruidStorageHandler.getHttpClient(), new Request(HttpMethod.GET, new URL(request))); + response = + DruidStorageHandlerUtils.submitRequest(DruidStorageHandler.getHttpClient(), + new Request(HttpMethod.GET, new URL(request))); } catch (Exception e) { throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); } @@ -266,8 +265,9 @@ // Retrieve results final List segmentDescriptors; try { - segmentDescriptors = DruidStorageHandlerUtils.JSON_MAPPER.readValue(response, - new TypeReference>() {}); + segmentDescriptors = + DruidStorageHandlerUtils.JSON_MAPPER.readValue(response, new TypeReference>() { + }); } catch (Exception e) { response.close(); throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); @@ -276,50 +276,39 @@ } private static String withQueryId(String druidQuery, String queryId) throws IOException { - Query queryWithId = - DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, BaseQuery.class).withId(queryId); + Query queryWithId = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, BaseQuery.class).withId(queryId); return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(queryWithId); } - @Override - public org.apache.hadoop.mapred.RecordReader getRecordReader( - org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter - ) - throws IOException { + @Override public org.apache.hadoop.mapred.RecordReader getRecordReader( + org.apache.hadoop.mapred.InputSplit split, + JobConf job, + Reporter reporter) 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 String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE); - if (druidQueryType == null) { - reader = new DruidScanQueryRecordReader(); // By default we use scan query as fallback. - reader.initialize((HiveDruidSplit) split, job); - return reader; - } - + // By default, we use druid scan query as fallback. + final String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE, Query.SCAN); reader = getDruidQueryReader(druidQueryType); - if (reader == null) { - throw new IOException("Druid query type " + druidQueryType + " not recognized"); - } reader.initialize((HiveDruidSplit) split, job); + if (Utilities.getIsVectorized(job)) { + //noinspection unchecked + return (org.apache.hadoop.mapred.RecordReader) new DruidVectorizedWrapper(reader, job); + } return reader; } - @Override - public RecordReader createRecordReader(InputSplit split, - TaskAttemptContext context - ) throws IOException, InterruptedException { + @Override public RecordReader createRecordReader(InputSplit split, + TaskAttemptContext context) throws IOException, InterruptedException { + // By default, we use druid scan query as fallback. + final String druidQueryType = context.getConfiguration().get(Constants.DRUID_QUERY_TYPE, Query.SCAN); // 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 String druidQueryType = context.getConfiguration().get(Constants.DRUID_QUERY_TYPE); - if (druidQueryType == null) { - return new DruidScanQueryRecordReader(); // By default, we use druid scan query as fallback. - } - final DruidQueryRecordReader reader = - getDruidQueryReader(druidQueryType); - if (reader == null) { - throw new IOException("Druid query type " + druidQueryType + " not recognized"); - } - return reader; + //noinspection unchecked + return getDruidQueryReader(druidQueryType); } + @Override public VectorizedSupport.Support[] getSupportedFeatures() { + return new VectorizedSupport.Support[0]; + } } 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 65edc665a3..15be0c325e 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 @@ -175,7 +175,7 @@ private SegmentIdentifier getSegmentIdentifierAndMaybePush(long truncatedTime) { private void pushSegments(List segmentsToPush) { try { SegmentsAndMetadata segmentsAndMetadata = appenderator.push(segmentsToPush, committerSupplier.get(), false).get(); - final HashSet pushedSegmentIdentifierHashSet = new HashSet<>(); + final Set pushedSegmentIdentifierHashSet = new HashSet<>(); for (DataSegment pushedSegment : segmentsAndMetadata.getSegments()) { pushedSegmentIdentifierHashSet.add(SegmentIdentifier.fromDataSegment(pushedSegment).getIdentifierAsString()); diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidVectorizedWrapper.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidVectorizedWrapper.java new file mode 100644 index 0000000000..cc95a02996 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidVectorizedWrapper.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.io; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader; +import org.apache.hadoop.hive.druid.serde.DruidSerDe; +import org.apache.hadoop.hive.druid.serde.DruidWritable; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.vector.VectorAssignRow; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.RecordReader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Properties; + +/** + * A Wrapper class that takes a row-by-row Druid Record Reader and provides a Vectorized one. + * @param type of the Druid query. + */ +public class DruidVectorizedWrapper> implements RecordReader { + private final VectorAssignRow vectorAssignRow = new VectorAssignRow(); + private final DruidQueryRecordReader baseReader; + private final VectorizedRowBatchCtx rbCtx; + private final DruidSerDe serDe; + + /** + * Actual projected columns needed by the query, this can be empty in case of query like: select count(*) from src. + */ + private final int[] projectedColumns; + + private final DruidWritable druidWritable; + + public DruidVectorizedWrapper(DruidQueryRecordReader reader, Configuration jobConf) { + this.rbCtx = Utilities.getVectorizedRowBatchCtx(jobConf); + if (rbCtx.getDataColumnNums() != null) { + projectedColumns = rbCtx.getDataColumnNums(); + } else { + // case all the columns are selected + projectedColumns = new int[rbCtx.getRowColumnTypeInfos().length]; + for (int i = 0; i < projectedColumns.length; i++) { + projectedColumns[i] = i; + } + } + this.serDe = createAndInitializeSerde(jobConf); + this.baseReader = Preconditions.checkNotNull(reader); + + // row parser and row assigner initializing + try { + vectorAssignRow.init((StructObjectInspector) serDe.getObjectInspector()); + } catch (HiveException e) { + throw new RuntimeException(e); + } + + druidWritable = baseReader.createValue(); + } + + @Override public boolean next(NullWritable nullWritable, VectorizedRowBatch vectorizedRowBatch) throws IOException { + vectorizedRowBatch.reset(); + ArrayList row; + int rowsCount = 0; + while (rowsCount < vectorizedRowBatch.getMaxSize() && baseReader.next(nullWritable, druidWritable)) { + if (projectedColumns.length > 0) { + try { + row = serDe.deserializeAsPrimitive(druidWritable); + } catch (SerDeException e) { + throw new IOException(e); + } + for (int i : projectedColumns) { + vectorAssignRow.assignRowColumn(vectorizedRowBatch, rowsCount, i, row.get(i)); + } + } + rowsCount++; + } + vectorizedRowBatch.size = rowsCount; + return rowsCount > 0; + } + + @Override public NullWritable createKey() { + return NullWritable.get(); + } + + @Override public VectorizedRowBatch createValue() { + return rbCtx.createVectorizedRowBatch(); + } + + @Override public long getPos() throws IOException { + return baseReader.getPos(); + } + + @Override public void close() throws IOException { + baseReader.close(); + } + + @Override public float getProgress() throws IOException { + return baseReader.getProgress(); + } + + private static DruidSerDe createAndInitializeSerde(Configuration jobConf) { + DruidSerDe serDe = new DruidSerDe(); + MapWork mapWork = Preconditions.checkNotNull(Utilities.getMapWork(jobConf), "Map work is null"); + Properties + properties = + mapWork.getPartitionDescs() + .stream() + .map(partitionDesc -> partitionDesc.getTableDesc().getProperties()) + .findAny() + .orElseThrow(() -> new RuntimeException("Can not find table property at the map work")); + try { + serDe.initialize(jobConf, properties, null); + } catch (SerDeException e) { + throw new RuntimeException("Can not initialized the serde", e); + } + return serDe; + } +} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/package-info.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/package-info.java new file mode 100644 index 0000000000..e5aac26f1b --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.druid.io; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroBytesDecoder.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroBytesDecoder.java index 3a1dbf7229..199987a9c1 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroBytesDecoder.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroBytesDecoder.java @@ -7,31 +7,25 @@ * "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 + * 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. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hive.druid.json; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.avro.generic.GenericRecord; -import java.nio.ByteBuffer; /** * This class is copied from druid source code * in order to avoid adding additional dependencies on druid-indexing-service. */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "schema_inline", value = InlineSchemaAvroBytesDecoder.class) -}) -public interface AvroBytesDecoder -{ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "schema_inline", value = InlineSchemaAvroBytesDecoder.class) }) +public interface AvroBytesDecoder { } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroParseSpec.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroParseSpec.java index af71f9a732..6cada1cb50 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroParseSpec.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroParseSpec.java @@ -7,14 +7,13 @@ * "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 + * 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. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hive.druid.json; @@ -34,55 +33,37 @@ * This class is copied from druid source code * in order to avoid adding additional dependencies on druid-indexing-service. */ -public class AvroParseSpec extends ParseSpec -{ +public class AvroParseSpec extends ParseSpec { - @JsonIgnore - private final JSONPathSpec flattenSpec; + @JsonIgnore private final JSONPathSpec flattenSpec; - @JsonCreator - public AvroParseSpec( - @JsonProperty("timestampSpec") TimestampSpec timestampSpec, + @JsonCreator public AvroParseSpec(@JsonProperty("timestampSpec") TimestampSpec timestampSpec, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, - @JsonProperty("flattenSpec") JSONPathSpec flattenSpec - ) - { - super( - timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null), - dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null) - ); + @JsonProperty("flattenSpec") JSONPathSpec flattenSpec) { + super(timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null), + dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null)); this.flattenSpec = flattenSpec != null ? flattenSpec : JSONPathSpec.DEFAULT; } - @JsonProperty - public JSONPathSpec getFlattenSpec() - { + @JsonProperty public JSONPathSpec getFlattenSpec() { return flattenSpec; } - @Override - public Parser makeParser() - { + @Override public Parser makeParser() { // makeParser is only used by StringInputRowParser, which cannot parse avro anyway. throw new UnsupportedOperationException("makeParser not supported"); } - @Override - public ParseSpec withTimestampSpec(TimestampSpec spec) - { + @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { return new AvroParseSpec(spec, getDimensionsSpec(), flattenSpec); } - @Override - public ParseSpec withDimensionsSpec(DimensionsSpec spec) - { + @Override public ParseSpec withDimensionsSpec(DimensionsSpec spec) { return new AvroParseSpec(getTimestampSpec(), spec, flattenSpec); } - @Override - public boolean equals(final Object o) - { + @Override public boolean equals(final Object o) { if (this == o) { return true; } @@ -96,9 +77,7 @@ public boolean equals(final Object o) return Objects.equals(flattenSpec, that.flattenSpec); } - @Override - public int hashCode() - { + @Override public int hashCode() { return Objects.hash(super.hashCode(), flattenSpec); } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroStreamInputRowParser.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroStreamInputRowParser.java index d6e6624669..9532e055d6 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroStreamInputRowParser.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/AvroStreamInputRowParser.java @@ -7,14 +7,13 @@ * "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 + * 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. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hive.druid.json; @@ -25,6 +24,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.ParseSpec; +import javax.validation.constraints.NotNull; import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; @@ -33,52 +33,33 @@ * This class is copied from druid source code * in order to avoid adding additional dependencies on druid-indexing-service. */ -public class AvroStreamInputRowParser implements ByteBufferInputRowParser -{ +public class AvroStreamInputRowParser implements ByteBufferInputRowParser { private final ParseSpec parseSpec; private final AvroBytesDecoder avroBytesDecoder; - @JsonCreator - public AvroStreamInputRowParser( - @JsonProperty("parseSpec") ParseSpec parseSpec, - @JsonProperty("avroBytesDecoder") AvroBytesDecoder avroBytesDecoder - ) - { + @JsonCreator public AvroStreamInputRowParser(@JsonProperty("parseSpec") ParseSpec parseSpec, + @JsonProperty("avroBytesDecoder") AvroBytesDecoder avroBytesDecoder) { this.parseSpec = Preconditions.checkNotNull(parseSpec, "parseSpec"); this.avroBytesDecoder = Preconditions.checkNotNull(avroBytesDecoder, "avroBytesDecoder"); } - @Override - public List parseBatch(ByteBuffer input) - { + @NotNull @Override public List parseBatch(ByteBuffer input) { throw new UnsupportedOperationException("This class is only used for JSON serde"); } - @JsonProperty - @Override - public ParseSpec getParseSpec() - { + @JsonProperty @Override public ParseSpec getParseSpec() { return parseSpec; } - @JsonProperty - public AvroBytesDecoder getAvroBytesDecoder() - { + @JsonProperty public AvroBytesDecoder getAvroBytesDecoder() { return avroBytesDecoder; } - @Override - public ByteBufferInputRowParser withParseSpec(ParseSpec parseSpec) - { - return new AvroStreamInputRowParser( - parseSpec, - avroBytesDecoder - ); + @Override public ByteBufferInputRowParser withParseSpec(ParseSpec parseSpec) { + return new AvroStreamInputRowParser(parseSpec, avroBytesDecoder); } - @Override - public boolean equals(final Object o) - { + @Override public boolean equals(final Object o) { if (this == o) { return true; } @@ -86,13 +67,10 @@ public boolean equals(final Object o) return false; } final AvroStreamInputRowParser that = (AvroStreamInputRowParser) o; - return Objects.equals(parseSpec, that.parseSpec) && - Objects.equals(avroBytesDecoder, that.avroBytesDecoder); + return Objects.equals(parseSpec, that.parseSpec) && Objects.equals(avroBytesDecoder, that.avroBytesDecoder); } - @Override - public int hashCode() - { + @Override public int hashCode() { return Objects.hash(parseSpec, avroBytesDecoder); } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/InlineSchemaAvroBytesDecoder.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/InlineSchemaAvroBytesDecoder.java index 72d6cbbc1e..1b6a08e746 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/InlineSchemaAvroBytesDecoder.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/InlineSchemaAvroBytesDecoder.java @@ -7,14 +7,13 @@ * "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 + * 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. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.hadoop.hive.druid.json; @@ -29,23 +28,16 @@ * This class is copied from druid source code * in order to avoid adding additional dependencies on druid-indexing-service. */ -public class InlineSchemaAvroBytesDecoder implements AvroBytesDecoder -{ +public class InlineSchemaAvroBytesDecoder implements AvroBytesDecoder { private final Map schema; - @JsonCreator - public InlineSchemaAvroBytesDecoder( - @JsonProperty("schema") Map schema - ) - { + @JsonCreator public InlineSchemaAvroBytesDecoder(@JsonProperty("schema") Map schema) { Preconditions.checkArgument(schema != null, "schema must be provided"); this.schema = schema; } - @JsonProperty - public Map getSchema() - { + @JsonProperty public Map getSchema() { return schema; } 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 c1b3bf8d41..fba591792d 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 @@ -43,8 +43,10 @@ private final Duration period; private final boolean useEarliestOffset; private final Duration completionTimeout; - @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private final Optional lateMessageRejectionPeriod; - @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private final Optional earlyMessageRejectionPeriod; + @SuppressWarnings({ "OptionalUsedAsFieldOrParameterType", "Guava" }) private final Optional + lateMessageRejectionPeriod; + @SuppressWarnings({ "OptionalUsedAsFieldOrParameterType", "Guava" }) private final Optional + earlyMessageRejectionPeriod; private final boolean skipOffsetGaps; @JsonCreator public KafkaSupervisorIOConfig(@JsonProperty("topic") String topic, @@ -71,10 +73,12 @@ this.period = defaultDuration(period, "PT30S"); this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false; this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); + //noinspection Guava this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null ? Optional.absent() : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); + //noinspection Guava this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null ? Optional.absent() : @@ -118,11 +122,11 @@ return completionTimeout; } - @JsonProperty public Optional getEarlyMessageRejectionPeriod() { + @SuppressWarnings("Guava") @JsonProperty public Optional getEarlyMessageRejectionPeriod() { return earlyMessageRejectionPeriod; } - @JsonProperty public Optional getLateMessageRejectionPeriod() { + @SuppressWarnings("Guava") @JsonProperty public Optional getLateMessageRejectionPeriod() { return lateMessageRejectionPeriod; } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java index 9ecba1b18c..9e09ddb45a 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/TaskReportData.java @@ -32,6 +32,9 @@ * in order to avoid adding additional dependencies on druid-indexing-service. */ public class TaskReportData { + /** + * Task type used by serializer but does not have any functionality as far i can tell. + */ public enum TaskType { ACTIVE, PUBLISHING, UNKNOWN } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/json/package-info.java druid-handler/src/java/org/apache/hadoop/hive/druid/json/package-info.java new file mode 100644 index 0000000000..dc53fa7de7 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/json/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.druid.json; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/package-info.java druid-handler/src/java/org/apache/hadoop/hive/druid/package-info.java new file mode 100644 index 0000000000..19acf64bb9 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.druid; diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/security/package-info.java druid-handler/src/java/org/apache/hadoop/hive/druid/security/package-info.java new file mode 100644 index 0000000000..5d8201e73a --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/security/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.druid.security; 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 53d74417f8..daebdcfa2d 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 @@ -80,25 +80,23 @@ initialize(split, context.getConfiguration()); } - public void initialize(InputSplit split, - Configuration conf, - ObjectMapper mapper, - ObjectMapper smileMapper, - HttpClient httpClient) throws IOException { + public void initialize(InputSplit split, ObjectMapper mapper, ObjectMapper smileMapper, HttpClient httpClient) + throws IOException { HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split; Preconditions.checkNotNull(hiveDruidSplit, "input split is null ???"); - ObjectMapper mapper1 = Preconditions.checkNotNull(mapper, "object Mapper can not be null"); + Preconditions.checkNotNull(httpClient, "need Http Client can not be null"); + ObjectMapper objectMapper = 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"); + ObjectMapper smileObjectMapper = Preconditions.checkNotNull(smileMapper, "Smile Mapper can not be null"); // Create query - this.query = mapper1.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class); + this.query = objectMapper.readValue(Preconditions.checkNotNull(hiveDruidSplit.getDruidQuery()), Query.class); Preconditions.checkNotNull(query); /* 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 initialized = false; int currentLocationIndex = 0; @@ -112,10 +110,14 @@ public void initialize(InputSplit split, LOG.debug("Retrieving data from druid location[{}] using query:[{}] ", address, query); try { Request request = DruidStorageHandlerUtils.createSmileRequest(address, query); - Future inputStreamFuture = httpClient1.go(request, new InputStreamResponseHandler()); + Future inputStreamFuture = httpClient.go(request, new InputStreamResponseHandler()); //noinspection unchecked queryResultsIterator = - new JsonParserIterator(smileMapper1, resultsType, inputStreamFuture, request.getUrl().toString(), query); + new JsonParserIterator(smileObjectMapper, + resultsType, + inputStreamFuture, + request.getUrl().toString(), + query); queryResultsIterator.init(); initialized = true; } catch (IOException | ExecutionException | InterruptedException e) { @@ -146,7 +148,6 @@ public void initialize(InputSplit split, public void initialize(InputSplit split, Configuration conf) throws IOException { initialize(split, - conf, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, DruidStorageHandler.getHttpClient()); @@ -175,8 +176,7 @@ public void initialize(InputSplit split, Configuration conf) throws IOException @Override public abstract NullWritable getCurrentKey() throws IOException, InterruptedException; - @Override - public abstract DruidWritable getCurrentValue() throws IOException, InterruptedException; + @Override public abstract DruidWritable getCurrentValue() throws IOException, InterruptedException; @Override public abstract float getProgress() throws IOException; 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 516faf0814..6213135227 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 @@ -405,9 +405,17 @@ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQ return null; } + /** + * This has the exact same logic as {@link this#deserializeAsPrimitive(Writable)}, any modification here should be + * done there as well. + * + * @param writable Druid Writable to be deserialized. + * @return List of Hive Writables. + * @throws SerDeException if there is Serde issues. + */ @Override public Object deserialize(Writable writable) throws SerDeException { final DruidWritable input = (DruidWritable) writable; - final List output = Lists.newArrayListWithExpectedSize(columns.length); + final ArrayList output = Lists.newArrayListWithExpectedSize(columns.length); for (int i = 0; i < columns.length; i++) { final Object value = input.isCompacted() ? input.getCompactedValue().get(i) : input.getValue().get(columns[i]); if (value == null) { @@ -474,7 +482,86 @@ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQ return output; } - private long deserializeToMillis(Object value) { + /** + * Function to convert Druid Primitive values to Hive Primitives. Main usage of this is to pipe data to VectorRow. + * This has the exact same logic as {@link this#deserialize(Writable)}, any modification here should be done there + * as well. + * Reason to have 2 function is performance, merging both will bring an extra test on the hot loop. + * + * @param writable Druid Writable container. + * @return ArrayList of Hive Primitives. + */ + public ArrayList deserializeAsPrimitive(Writable writable) throws SerDeException { + final DruidWritable input = (DruidWritable) writable; + final ArrayList output = Lists.newArrayListWithExpectedSize(columns.length); + + for (int i = 0; i < columns.length; i++) { + final Object value = input.isCompacted() ? input.getCompactedValue().get(i) : input.getValue().get(columns[i]); + if (value == null) { + output.add(null); + continue; + } + switch (types[i].getPrimitiveCategory()) { + case TIMESTAMP: + output.add(Timestamp.ofEpochMilli(deserializeToMillis(value))); + break; + case TIMESTAMPLOCALTZ: + output.add(new TimestampTZ(ZonedDateTime.ofInstant(Instant.ofEpochMilli(deserializeToMillis(value)), + ((TimestampLocalTZTypeInfo) types[i]).timeZone()))); + break; + case DATE: + output.add(Date.ofEpochMilli(deserializeToMillis(value))); + break; + case BYTE: + output.add(((Number) value).byteValue()); + break; + case SHORT: + output.add(((Number) value).shortValue()); + break; + case INT: + if (value instanceof Number) { + output.add(((Number) value).intValue()); + } else { + // This is a corner case where we have an extract of time unit like day/month pushed as Extraction Fn + //@TODO The best way to fix this is to add explicit output Druid types to Calcite Extraction Functions impls + output.add(Integer.valueOf((String) value)); + } + + break; + case LONG: + output.add(((Number) value).longValue()); + break; + case FLOAT: + output.add(((Number) value).floatValue()); + break; + case DOUBLE: + output.add(((Number) value).doubleValue()); + break; + case CHAR: + output.add(new HiveChar(value.toString(), ((CharTypeInfo) types[i]).getLength())); + break; + case VARCHAR: + output.add(new HiveVarchar(value.toString(), ((VarcharTypeInfo) types[i]).getLength())); + break; + case STRING: + output.add(value.toString()); + break; + case BOOLEAN: + if (value instanceof Number) { + output.add(((Number) value).intValue() != 0); + } else { + output.add(Boolean.valueOf(value.toString())); + } + break; + default: + throw new SerDeException("Unknown type: " + types[i].getPrimitiveCategory()); + } + } + + return output; + } + + private static long deserializeToMillis(Object value) { long numberOfMillis; if (value instanceof Number) { numberOfMillis = ((Number) value).longValue(); diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/package-info.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/package-info.java new file mode 100644 index 0000000000..ba6a103af1 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.druid.serde; 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 bf42a74f0f..da05b832d2 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/DerbyConnectorTestUtility.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -32,6 +32,9 @@ import java.util.UUID; import java.util.function.Supplier; +/** + * Derby test class. + */ public class DerbyConnectorTestUtility extends DerbyConnector { private final String jdbcUri; @@ -71,6 +74,9 @@ public String getJdbcUri() { return jdbcUri; } + /** + * Derby connector rule. + */ public static class DerbyConnectorRule extends ExternalResource { private DerbyConnectorTestUtility connector; diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java index 099e5b3357..64d5b2ec3e 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java @@ -45,26 +45,41 @@ + " \"id\" : \"merged\",\r\n " + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n " + " \"columns\" : {\r\n " - + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, \"cardinality\" : 1504, \"errorMessage\" : null },\r\n " - + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"unpatrolled\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null }\r\n " + + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, " + + "\"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\"" + + " : 1944, \"errorMessage\" : null },\r\n " + + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, " + + "\"cardinality\" : 1504, \"errorMessage\" : null },\r\n " + + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"unpatrolled\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": 1944, \"errorMessage\" : null },\r\n " + + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": 1944, \"errorMessage\" : null },\r\n " + + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": null, \"errorMessage\" : null },\r\n " + + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": null, \"errorMessage\" : null },\r\n " + + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": null, \"errorMessage\" : null },\r\n " + + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : null, \"errorMessage\" : null }\r\n " + " },\r\n " + " \"aggregators\" : {\r\n " + " \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n " + " \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n " + " \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n " - + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" },\r\n " + + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" }," + + "\r\n " + " \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n " + " },\r\n " + " \"queryGranularity\" : {\r\n \"type\": \"none\"\r\n },\r\n " diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java index f52e721763..35808c9d29 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe2.java @@ -39,50 +39,64 @@ // + "\"intervals\":[\"-146136543-09-08T00:30:34.096-07:52:58/146140482-04-24T08:36:27.903-07:00\"]}," // + "\"toInclude\":{\"type\":\"all\"},\"merge\":true,\"context\":null,\"analysisTypes\":[]," // + "\"usingDefaultInterval\":true,\"lenientAggregatorMerge\":false,\"descending\":false}"; - private static final String RESPONSE = - "[ {\r\n " - + " \"id\" : \"merged\",\r\n " - + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n " - + " \"columns\" : {\r\n " - + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, \"cardinality\" : 1504, \"errorMessage\" : null },\r\n " - + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - // Next column has a similar name as previous, but different casing. - // This is allowed in Druid, but it should fail in Hive. - + " \"Anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null }\r\n " - + " },\r\n " - + " \"aggregators\" : {\r\n " - + " \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n " - + " \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n " - + " \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n " - + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" },\r\n " - + " \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n " - + " },\r\n " - + " \"queryGranularity\" : {\r\n \"type\": \"none\"\r\n },\r\n " - + " \"size\" : 300000,\r\n " - + " \"numRows\" : 5000000\r\n} ]"; + private static final String + RESPONSE = + "[ {\r\n " + + " \"id\" : \"merged\",\r\n " + + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n " + + " \"columns\" : {\r\n " + + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, " + + "\"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\"" + + " : 1944, \"errorMessage\" : null },\r\n " + + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, " + + "\"cardinality\" : 1504, \"errorMessage\" : null },\r\n " + + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + // Next column has a similar name as previous, but different casing. + // This is allowed in Druid, but it should fail in Hive. + + " \"Anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": 1944, \"errorMessage\" : null },\r\n " + + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": 1944, \"errorMessage\" : null },\r\n " + + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": null, \"errorMessage\" : null },\r\n " + + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": null, \"errorMessage\" : null },\r\n " + + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" " + + ": null, \"errorMessage\" : null },\r\n " + + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, " + + "\"cardinality\" : null, \"errorMessage\" : null }\r\n " + + " },\r\n " + + " \"aggregators\" : {\r\n " + + " \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n " + + " \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n " + + " \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n " + + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" }," + + "\r\n " + + " \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n " + + " },\r\n " + + " \"queryGranularity\" : {\r\n \"type\": \"none\"\r\n },\r\n " + + " \"size\" : 300000,\r\n " + + " \"numRows\" : 5000000\r\n} ]"; /* Submits the request and returns */ - @Override - protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query) - throws SerDeException { + @Override protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query) + throws SerDeException { // Retrieve results List resultsList; try { - resultsList = DruidStorageHandlerUtils.JSON_MAPPER.readValue(RESPONSE, - new TypeReference>() { - } - ); + resultsList = + DruidStorageHandlerUtils.JSON_MAPPER.readValue(RESPONSE, new TypeReference>() { + }); } catch (Exception e) { throw new SerDeException(StringUtils.stringifyException(e)); } 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 0cb3c237b1..419f088601 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidStorageHandler.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -60,15 +60,19 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.UUID; +/** + * Test class for Druid Storage handler. + */ public class TestDruidStorageHandler { - @Rule - public final DerbyConnectorTestUtility.DerbyConnectorRule derbyConnectorRule = new DerbyConnectorTestUtility.DerbyConnectorRule(); + @Rule public final DerbyConnectorTestUtility.DerbyConnectorRule + derbyConnectorRule = + new DerbyConnectorTestUtility.DerbyConnectorRule(); - @Rule - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); private static final String DB_NAME = "default"; private static final String TABLE_NAME = "testName"; @@ -86,19 +90,22 @@ private DataSegment createSegment(String location) throws IOException { return createSegment(location, new Interval(100, 170, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); } - private DataSegment createSegment(String location, Interval interval, String version, - ShardSpec shardSpec) throws IOException { + private DataSegment createSegment(String location, Interval interval, String version, ShardSpec shardSpec) + throws IOException { FileUtils.writeStringToFile(new File(location), "dummySegmentData"); - return DataSegment.builder().dataSource(DATA_SOURCE_NAME).version(version) - .interval(interval).shardSpec(shardSpec) - .loadSpec(ImmutableMap.of("path", location)).build(); + return DataSegment.builder() + .dataSource(DATA_SOURCE_NAME) + .version(version) + .interval(interval) + .shardSpec(shardSpec) + .loadSpec(ImmutableMap.of("path", location)) + .build(); } - @Before - public void before() throws Throwable { + @Before public void before() throws Throwable { tableWorkingPath = temporaryFolder.newFolder().getAbsolutePath(); segmentsTable = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); - HashMap params = new HashMap<>(); + Map params = new HashMap<>(); params.put("external.table.purge", "TRUE"); Mockito.when(tableMock.getParameters()).thenReturn(params); Mockito.when(tableMock.getPartitionKeysSize()).thenReturn(0); @@ -111,179 +118,157 @@ public void before() throws Throwable { config.set(String.valueOf(HiveConf.ConfVars.HIVEQUERYID), "hive-" + UUID.randomUUID().toString()); config.set(String.valueOf(HiveConf.ConfVars.DRUID_WORKING_DIR), tableWorkingPath); config.set(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY), - new Path(tableWorkingPath, "finalSegmentDir").toString()); + new Path(tableWorkingPath, "finalSegmentDir").toString()); config.set("hive.druid.maxTries", "0"); - druidStorageHandler = new DruidStorageHandler( - derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - ); + druidStorageHandler = + new DruidStorageHandler(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get()); druidStorageHandler.setConf(config); } - @After - public void tearDown() { + @After public void tearDown() { temporaryFolder.delete(); } private final Table tableMock = Mockito.mock(Table.class); - @Test - public void testPreCreateTableWillCreateSegmentsTable() throws MetaException { + @Test public void testPreCreateTableWillCreateSegmentsTable() throws MetaException { try (Handle handle = derbyConnectorRule.getConnector().getDBI().open()) { - Assert.assertFalse(derbyConnectorRule.getConnector() - .tableExists(handle, - segmentsTable - )); + Assert.assertFalse(derbyConnectorRule.getConnector().tableExists(handle, segmentsTable)); druidStorageHandler.preCreateTable(tableMock); - Assert.assertTrue(derbyConnectorRule.getConnector() - .tableExists(handle, - segmentsTable - )); + Assert.assertTrue(derbyConnectorRule.getConnector().tableExists(handle, segmentsTable)); } } - @Test(expected = MetaException.class) - public void testPreCreateTableWhenDataSourceExists() throws MetaException, IOException { + @Test(expected = MetaException.class) public void testPreCreateTableWhenDataSourceExists() + throws MetaException, IOException { derbyConnectorRule.getConnector().createSegmentTable(); - SQLMetadataStorageUpdaterJobHandler sqlMetadataStorageUpdaterJobHandler = new SQLMetadataStorageUpdaterJobHandler( - derbyConnectorRule.getConnector()); + SQLMetadataStorageUpdaterJobHandler + sqlMetadataStorageUpdaterJobHandler = + new SQLMetadataStorageUpdaterJobHandler(derbyConnectorRule.getConnector()); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); DataSegment dataSegment = createSegment(new Path(taskDirPath, "intermediatePath").toString()); - sqlMetadataStorageUpdaterJobHandler.publishSegments(segmentsTable, Collections.singletonList(dataSegment), - DruidStorageHandlerUtils.JSON_MAPPER - ); + sqlMetadataStorageUpdaterJobHandler.publishSegments(segmentsTable, + Collections.singletonList(dataSegment), + DruidStorageHandlerUtils.JSON_MAPPER); druidStorageHandler.preCreateTable(tableMock); } - @Test - public void testCommitCreateTablePlusCommitDropTableWithoutPurge() - throws MetaException, IOException { + @Test public void testCommitCreateTablePlusCommitDropTableWithoutPurge() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); DataSegment dataSegment = createSegment(new Path(taskDirPath, "index.zip").toString()); - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); druidStorageHandler.commitCreateTable(tableMock); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); druidStorageHandler.commitDropTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); } - @Test - public void testCommitCreateTablePlusCommitDropTableWithPurge() - throws MetaException, IOException { + @Test public void testCommitCreateTablePlusCommitDropTableWithPurge() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); DataSegment dataSegment = createSegment(new Path(taskDirPath, "index.zip").toString()); - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); druidStorageHandler.commitCreateTable(tableMock); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); druidStorageHandler.commitDropTable(tableMock, true); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); } - @Test - public void testCommitCreateEmptyTablePlusCommitDropTableWithoutPurge() - throws MetaException, IOException { + @Test public void testCommitCreateEmptyTablePlusCommitDropTableWithoutPurge() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); druidStorageHandler.commitCreateTable(tableMock); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); druidStorageHandler.commitDropTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); } - @Test - public void testCommitCreateEmptyTablePlusCommitDropTableWithPurge() - throws MetaException, IOException { + @Test public void testCommitCreateEmptyTablePlusCommitDropTableWithPurge() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); druidStorageHandler.commitCreateTable(tableMock); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); druidStorageHandler.commitDropTable(tableMock, true); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); } - @Test - public void testCommitInsertTable() throws MetaException, IOException { + @Test public void testCommitInsertTable() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); DataSegment dataSegment = createSegment(new Path(taskDirPath, "index.zip").toString()); - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); druidStorageHandler.commitCreateTable(tableMock); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); } - @Test - public void testCommitEmptyInsertTable() throws MetaException, IOException { + @Test public void testCommitEmptyInsertTable() throws MetaException, IOException { druidStorageHandler.preCreateTable(tableMock); druidStorageHandler.commitCreateTable(tableMock); - Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), - derbyConnectorRule.metadataTablesConfigSupplier().get() - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get())).toArray()); } - @Test - public void testDeleteSegment() throws IOException, SegmentLoadingException { + @Test public void testDeleteSegment() throws IOException, SegmentLoadingException { String segmentRootPath = temporaryFolder.newFolder().getAbsolutePath(); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); DataSegment dataSegment = createSegment(new Path(taskDirPath, "index.zip").toString()); HdfsDataSegmentPusherConfig hdfsDSPConfig = new HdfsDataSegmentPusherConfig(); hdfsDSPConfig.setStorageDirectory(segmentRootPath); - HdfsDataSegmentPusher hdfsDataSegmentPusher = new HdfsDataSegmentPusher(hdfsDSPConfig, config, - DruidStorageHandlerUtils.JSON_MAPPER - ); - Path segmentOutputPath = JobHelper - .makeFileNamePath(new Path(segmentRootPath), localFileSystem, dataSegment, - JobHelper.INDEX_ZIP, hdfsDataSegmentPusher - ); + HdfsDataSegmentPusher + hdfsDataSegmentPusher = + new HdfsDataSegmentPusher(hdfsDSPConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + Path + segmentOutputPath = + JobHelper.makeFileNamePath(new Path(segmentRootPath), + localFileSystem, + dataSegment, + JobHelper.INDEX_ZIP, + hdfsDataSegmentPusher); Path indexPath = new Path(segmentOutputPath, "index.zip"); - DataSegment dataSegmentWithLoadspect = DataSegment.builder(dataSegment).loadSpec( - ImmutableMap.of("path", indexPath)).build(); + DataSegment + dataSegmentWithLoadspect = + DataSegment.builder(dataSegment).loadSpec(ImmutableMap.of("path", indexPath)).build(); OutputStream outputStream = localFileSystem.create(indexPath, true); outputStream.close(); Assert.assertTrue("index file is not created ??", localFileSystem.exists(indexPath)); @@ -293,105 +278,104 @@ public void testDeleteSegment() throws IOException, SegmentLoadingException { // path format -- > .../dataSource/interval/version/partitionNum/xxx.zip Assert.assertFalse("Index file still there ??", localFileSystem.exists(indexPath)); // path format of segmentOutputPath -- > .../dataSource/interval/version/partitionNum/ - Assert.assertFalse("PartitionNum directory still there ??", - localFileSystem.exists(segmentOutputPath) - ); - Assert.assertFalse("Version directory still there ??", - localFileSystem.exists(segmentOutputPath.getParent()) - ); + Assert.assertFalse("PartitionNum directory still there ??", localFileSystem.exists(segmentOutputPath)); + Assert.assertFalse("Version directory still there ??", localFileSystem.exists(segmentOutputPath.getParent())); Assert.assertFalse("Interval directory still there ??", - localFileSystem.exists(segmentOutputPath.getParent().getParent()) - ); + localFileSystem.exists(segmentOutputPath.getParent().getParent())); Assert.assertFalse("Data source directory still there ??", - localFileSystem.exists(segmentOutputPath.getParent().getParent().getParent()) - ); + localFileSystem.exists(segmentOutputPath.getParent().getParent().getParent())); } - @Test - public void testCommitInsertOverwriteTable() throws MetaException, IOException { + @Test public void testCommitInsertOverwriteTable() throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(config.get(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY))); - DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); // This create and publish the segment to be overwritten - List existingSegments = + 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, - true, - config, - dataSegmentPusher - ); + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); // This creates and publish new segment - DataSegment dataSegment = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(180, 250, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DataSegment + dataSegment = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(180, 250, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); druidStorageHandler.commitInsertTable(tableMock, true); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); - final List dataSegmentList = getUsedSegmentsList(connector, - metadataStorageTablesConfig); + final List dataSegmentList = getUsedSegmentsList(connector, metadataStorageTablesConfig); Assert.assertEquals(1, dataSegmentList.size()); DataSegment persistedSegment = Iterables.getOnlyElement(dataSegmentList); Assert.assertEquals(dataSegment, persistedSegment); Assert.assertEquals(dataSegment.getVersion(), persistedSegment.getVersion()); - Path expectedFinalHadoopPath = new Path(dataSegmentPusher.getPathForHadoop(), dataSegmentPusher - .makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); + Path + expectedFinalHadoopPath = + new Path(dataSegmentPusher.getPathForHadoop(), + dataSegmentPusher.makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); Assert.assertEquals(ImmutableMap.of("type", "hdfs", "path", expectedFinalHadoopPath.toString()), - persistedSegment.getLoadSpec()); - Assert.assertEquals("dummySegmentData", - FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); + persistedSegment.getLoadSpec()); + Assert.assertEquals("dummySegmentData", FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); } - @Test - public void testCommitMultiInsertOverwriteTable() throws MetaException, IOException { + @Test public void testCommitMultiInsertOverwriteTable() throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); LocalFileSystem localFileSystem = FileSystem.getLocal(config); druidStorageHandler.preCreateTable(tableMock); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(config.get(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY))); - DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); // This create and publish the segment to be overwritten - List existingSegments = + 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, - true, - config, - dataSegmentPusher - ); + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); // Check that there is one datasource with the published segment - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); // Sequence is the following: // 1) INSERT with no segments -> Original segment still present in the datasource @@ -406,136 +390,139 @@ public void testCommitMultiInsertOverwriteTable() throws MetaException, IOExcept // We start: // #1 druidStorageHandler.commitInsertTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); - Assert.assertEquals(1, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); + Assert.assertEquals(1, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #2 druidStorageHandler.commitInsertTable(tableMock, true); - Assert.assertEquals(0, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertEquals(0, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #3 druidStorageHandler.commitInsertTable(tableMock, true); - Assert.assertEquals(0, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertEquals(0, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #4 druidStorageHandler.commitInsertTable(tableMock, true); - Assert.assertEquals(0, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertEquals(0, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #5 - DataSegment dataSegment1 = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(180, 250, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath1 = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment1, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DataSegment + dataSegment1 = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(180, 250, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath1 = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment1, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment1, descriptorPath1); druidStorageHandler.commitInsertTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); - Assert.assertEquals(1, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); + Assert.assertEquals(1, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #6 - DataSegment dataSegment2 = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(200, 250, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath2 = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment2, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DataSegment + dataSegment2 = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(200, 250, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath2 = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment2, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment2, descriptorPath2); druidStorageHandler.commitInsertTable(tableMock, true); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); - Assert.assertEquals(1, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); + Assert.assertEquals(1, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #7 - DataSegment dataSegment3 = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 200, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath3 = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment3, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DataSegment + dataSegment3 = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 200, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath3 = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment3, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment3, descriptorPath3); druidStorageHandler.commitInsertTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); - Assert.assertEquals(2, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); + Assert.assertEquals(2, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); // #8 druidStorageHandler.commitInsertTable(tableMock, true); - Assert.assertEquals(0, getUsedSegmentsList(connector, - metadataStorageTablesConfig).size()); + Assert.assertEquals(0, getUsedSegmentsList(connector, metadataStorageTablesConfig).size()); } private List getUsedSegmentsList(DerbyConnectorTestUtility connector, - final MetadataStorageTablesConfig metadataStorageTablesConfig) { + final MetadataStorageTablesConfig metadataStorageTablesConfig) { return connector.getDBI() - .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()); + .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 - public void testCommitInsertIntoTable() throws MetaException, IOException { + @Test public void testCommitInsertIntoTable() throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); - List existingSegments = + 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); - - DruidStorageHandlerUtils - .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, - existingSegments, - true, - config, - dataSegmentPusher - ); - DataSegment dataSegment = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); + DataSegment + dataSegment = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); druidStorageHandler.commitInsertTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); - final List dataSegmentList = getUsedSegmentsList(connector, - metadataStorageTablesConfig); + final List dataSegmentList = getUsedSegmentsList(connector, metadataStorageTablesConfig); Assert.assertEquals(2, dataSegmentList.size()); DataSegment persistedSegment = dataSegmentList.get(1); @@ -544,45 +531,52 @@ public void testCommitInsertIntoTable() throws MetaException, IOException { Assert.assertTrue(persistedSegment.getShardSpec() instanceof LinearShardSpec); Assert.assertEquals(2, persistedSegment.getShardSpec().getPartitionNum()); - Path expectedFinalHadoopPath = new Path(dataSegmentPusher.getPathForHadoop(), dataSegmentPusher - .makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); + Path + expectedFinalHadoopPath = + new Path(dataSegmentPusher.getPathForHadoop(), + dataSegmentPusher.makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); Assert.assertEquals(ImmutableMap.of("type", "hdfs", "path", expectedFinalHadoopPath.toString()), - persistedSegment.getLoadSpec()); - Assert.assertEquals("dummySegmentData", - FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); + persistedSegment.getLoadSpec()); + Assert.assertEquals("dummySegmentData", FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); } - @Test - public void testInsertIntoAppendOneMorePartition() throws MetaException, IOException { + @Test public void testInsertIntoAppendOneMorePartition() throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(config.get(String.valueOf(HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY))); - DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); - List existingSegments = + 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, - true, - config, - dataSegmentPusher - ); - - DataSegment dataSegment = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v0", new LinearShardSpec(0)); - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); + + DataSegment + dataSegment = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0)); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); druidStorageHandler.commitInsertTable(tableMock, false); Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( @@ -590,8 +584,7 @@ public void testInsertIntoAppendOneMorePartition() throws MetaException, IOExcep metadataStorageTablesConfig )).toArray()); - final List dataSegmentList = getUsedSegmentsList(connector, - metadataStorageTablesConfig); + final List dataSegmentList = getUsedSegmentsList(connector, metadataStorageTablesConfig); Assert.assertEquals(2, dataSegmentList.size()); DataSegment persistedSegment = dataSegmentList.get(1); @@ -599,60 +592,71 @@ public void testInsertIntoAppendOneMorePartition() throws MetaException, IOExcep Assert.assertTrue(persistedSegment.getShardSpec() instanceof LinearShardSpec); Assert.assertEquals(1, persistedSegment.getShardSpec().getPartitionNum()); - Path expectedFinalHadoopPath = new Path(dataSegmentPusher.getPathForHadoop(), dataSegmentPusher - .makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); + Path + expectedFinalHadoopPath = + new Path(dataSegmentPusher.getPathForHadoop(), + dataSegmentPusher.makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); Assert.assertEquals(ImmutableMap.of("type", "hdfs", "path", expectedFinalHadoopPath.toString()), - persistedSegment.getLoadSpec()); - Assert.assertEquals("dummySegmentData", - FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); + persistedSegment.getLoadSpec()); + Assert.assertEquals("dummySegmentData", FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); } - @Test - public void testCommitInsertIntoWhenDestinationSegmentFileExist() - throws MetaException, IOException { + @Test public void testCommitInsertIntoWhenDestinationSegmentFileExist() throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); druidStorageHandler.preCreateTable(tableMock); LocalFileSystem localFileSystem = FileSystem.getLocal(config); Path taskDirPath = new Path(tableWorkingPath, druidStorageHandler.makeStagingName()); - List existingSegments = + 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); - DruidStorageHandlerUtils - .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, - existingSegments, - true, - config, - dataSegmentPusher - ); - DataSegment dataSegment = createSegment(new Path(taskDirPath, "index.zip").toString(), - new Interval(100, 150, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); + DataSegment + dataSegment = + createSegment(new Path(taskDirPath, "index.zip").toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); // Create segment file at the destination location with LinearShardSpec(2) - DataSegment segment = createSegment(new Path(taskDirPath, "index_conflict.zip").toString(), - new Interval(100, 150, DateTimeZone.UTC), "v1", new LinearShardSpec(1)); - Path segmentPath = new Path(dataSegmentPusher.getPathForHadoop(), dataSegmentPusher.makeIndexPathName(segment, DruidStorageHandlerUtils.INDEX_ZIP)); + DataSegment + segment = + createSegment(new Path(taskDirPath, "index_conflict.zip").toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v1", + new LinearShardSpec(1)); + Path + segmentPath = + new Path(dataSegmentPusher.getPathForHadoop(), + dataSegmentPusher.makeIndexPathName(segment, DruidStorageHandlerUtils.INDEX_ZIP)); FileUtils.writeStringToFile(new File(segmentPath.toUri()), "dummy"); druidStorageHandler.commitInsertTable(tableMock, false); - Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( - DruidStorageHandlerUtils.getAllDataSourceNames(connector, - metadataStorageTablesConfig - )).toArray()); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), + Lists.newArrayList(DruidStorageHandlerUtils.getAllDataSourceNames(connector, metadataStorageTablesConfig)) + .toArray()); - final List dataSegmentList = getUsedSegmentsList(connector, - metadataStorageTablesConfig); + final List dataSegmentList = getUsedSegmentsList(connector, metadataStorageTablesConfig); Assert.assertEquals(2, dataSegmentList.size()); DataSegment persistedSegment = dataSegmentList.get(1); @@ -661,93 +665,111 @@ public void testCommitInsertIntoWhenDestinationSegmentFileExist() Assert.assertTrue(persistedSegment.getShardSpec() instanceof LinearShardSpec); // insert into should skip and increment partition number to 3 Assert.assertEquals(2, persistedSegment.getShardSpec().getPartitionNum()); - Path expectedFinalHadoopPath = new Path(dataSegmentPusher.getPathForHadoop(), dataSegmentPusher - .makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); - + Path + expectedFinalHadoopPath = + new Path(dataSegmentPusher.getPathForHadoop(), + dataSegmentPusher.makeIndexPathName(persistedSegment, DruidStorageHandlerUtils.INDEX_ZIP)); Assert.assertEquals(ImmutableMap.of("type", "hdfs", "path", expectedFinalHadoopPath.toString()), - persistedSegment.getLoadSpec()); - Assert.assertEquals("dummySegmentData", - FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); + persistedSegment.getLoadSpec()); + Assert.assertEquals("dummySegmentData", FileUtils.readFileToString(new File(expectedFinalHadoopPath.toUri()))); } - @Test(expected = MetaException.class) - public void testCommitInsertIntoWithConflictingIntervalSegment() - throws MetaException, IOException { + @Test(expected = MetaException.class) public void testCommitInsertIntoWithConflictingIntervalSegment() + throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); 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_1.zip").toString(), - new Interval(100, 150, DateTimeZone.UTC), - "v0", new LinearShardSpec(0)), + List + existingSegments = + Arrays.asList(createSegment(new Path(taskDirPath, "index_old_1.zip").toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0)), createSegment(new Path(taskDirPath, "index_old_2.zip").toString(), - new Interval(150, 200, DateTimeZone.UTC), - "v0", new LinearShardSpec(0)), + new Interval(150, 200, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0)), createSegment(new Path(taskDirPath, "index_old_3.zip").toString(), - new Interval(200, 300, DateTimeZone.UTC), - "v0", new LinearShardSpec(0))); + new Interval(200, 300, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0))); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(taskDirPath.toString()); - DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); - DruidStorageHandlerUtils - .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, - existingSegments, - true, - config, - dataSegmentPusher - ); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); // Try appending segment with conflicting interval - DataSegment conflictingSegment = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 300, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath = DruidStorageHandlerUtils - .makeSegmentDescriptorOutputPath(conflictingSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); - DruidStorageHandlerUtils - .writeSegmentDescriptor(localFileSystem, conflictingSegment, descriptorPath); + DataSegment + conflictingSegment = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 300, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(conflictingSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); + DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, conflictingSegment, descriptorPath); druidStorageHandler.commitInsertTable(tableMock, false); } - @Test(expected = MetaException.class) - public void testCommitInsertIntoWithNonExtendableSegment() throws MetaException, IOException { + @Test(expected = MetaException.class) public void testCommitInsertIntoWithNonExtendableSegment() + throws MetaException, IOException { DerbyConnectorTestUtility connector = derbyConnectorRule.getConnector(); - MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule - .metadataTablesConfigSupplier().get(); + MetadataStorageTablesConfig metadataStorageTablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); 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_1.zip").toString(), - 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(), - new Interval(250, 300, DateTimeZone.UTC), "v0", new LinearShardSpec(0))); + List + existingSegments = + Arrays.asList(createSegment(new Path(taskDirPath, "index_old_1.zip").toString(), + 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(), + new Interval(250, 300, DateTimeZone.UTC), + "v0", + new LinearShardSpec(0))); HdfsDataSegmentPusherConfig pusherConfig = new HdfsDataSegmentPusherConfig(); pusherConfig.setStorageDirectory(taskDirPath.toString()); - DataSegmentPusher dataSegmentPusher = new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); - DruidStorageHandlerUtils - .publishSegmentsAndCommit(connector, metadataStorageTablesConfig, DATA_SOURCE_NAME, - existingSegments, - true, - config, - dataSegmentPusher - ); + DataSegmentPusher + dataSegmentPusher = + new HdfsDataSegmentPusher(pusherConfig, config, DruidStorageHandlerUtils.JSON_MAPPER); + DruidStorageHandlerUtils.publishSegmentsAndCommit(connector, + metadataStorageTablesConfig, + DATA_SOURCE_NAME, + existingSegments, + true, + config, + dataSegmentPusher); // Try appending to non extendable shard spec - DataSegment conflictingSegment = createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), - new Interval(100, 150, DateTimeZone.UTC), "v1", new LinearShardSpec(0)); - Path descriptorPath = DruidStorageHandlerUtils - .makeSegmentDescriptorOutputPath(conflictingSegment, - new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) - ); - DruidStorageHandlerUtils - .writeSegmentDescriptor(localFileSystem, conflictingSegment, descriptorPath); + DataSegment + conflictingSegment = + createSegment(new Path(taskDirPath, DruidStorageHandlerUtils.INDEX_ZIP).toString(), + new Interval(100, 150, DateTimeZone.UTC), + "v1", + new LinearShardSpec(0)); + Path + descriptorPath = + DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(conflictingSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME)); + DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, conflictingSegment, descriptorPath); druidStorageHandler.commitInsertTable(tableMock, false); 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 513119ea32..42bde3583e 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java @@ -30,18 +30,24 @@ import io.druid.query.Query; import junit.framework.TestCase; +/** + * Test Class. + */ @SuppressWarnings("SameParameterValue") public class TestHiveDruidQueryBasedInputFormat extends TestCase { - private static final String TIMESERIES_QUERY = + private static final String + TIMESERIES_QUERY = "{ \"queryType\": \"timeseries\", " + " \"dataSource\": \"sample_datasource\", " + " \"granularity\": \"DAY\", " + " \"descending\": \"true\", " + " \"intervals\": [ \"2012-01-01T00:00:00.000-08:00/2012-01-03T00:00:00.000-08:00\" ]}"; - private static final String TIMESERIES_QUERY_SPLIT = + private static final String + TIMESERIES_QUERY_SPLIT = "[HiveDruidSplit{{\"queryType\":\"timeseries\"," + "\"dataSource\":{\"type\":\"table\",\"name\":\"sample_datasource\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2012-01-01T08:00:00.000Z/2012-01-03T08:00:00.000Z\"]}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":" + + "[\"2012-01-01T08:00:00.000Z/2012-01-03T08:00:00.000Z\"]}," + "\"descending\":true," + "\"virtualColumns\":[]," + "\"filter\":null," @@ -50,7 +56,8 @@ + "\"postAggregations\":[]," + "\"context\":{\"queryId\":\"\"}}, [localhost:8082]}]"; - private static final String TOPN_QUERY = + private static final String + TOPN_QUERY = "{ \"queryType\": \"topN\", " + " \"dataSource\": \"sample_data\", " + " \"dimension\": \"sample_dim\", " @@ -72,14 +79,17 @@ + " \"intervals\": [ " + " \"2013-08-31T00:00:00.000-07:00/2013-09-03T00:00:00.000-07:00\" " + " ]}"; - private static final String TOPN_QUERY_SPLIT = + private static final String + TOPN_QUERY_SPLIT = "[HiveDruidSplit{{\"queryType\":\"topN\"," + "\"dataSource\":{\"type\":\"table\",\"name\":\"sample_data\"}," + "\"virtualColumns\":[]," - + "\"dimension\":{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"sample_dim\",\"outputName\":\"sample_dim\",\"outputType\":\"STRING\"}," + + "\"dimension\":{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"sample_dim\"," + + "\"outputName\":\"sample_dim\",\"outputType\":\"STRING\"}," + "\"metric\":{\"type\":\"LegacyTopNMetricSpec\",\"metric\":\"count\"}," + "\"threshold\":5," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2013-08-31T07:00:00.000Z/2013-09-03T07:00:00.000Z\"]}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2013-08-31T07:00:00" + + ".000Z/2013-09-03T07:00:00.000Z\"]}," + "\"filter\":null," + "\"granularity\":{\"type\":\"all\"}," + "\"aggregations\":[{\"type\":\"longSum\",\"name\":\"count\",\"fieldName\":\"count\",\"expression\":null}," @@ -88,7 +98,8 @@ + "\"context\":{\"queryId\":\"\"}," + "\"descending\":false}, [localhost:8082]}]"; - private static final String GROUP_BY_QUERY = + private static final String + GROUP_BY_QUERY = "{ \"queryType\": \"groupBy\", " + " \"dataSource\": \"sample_datasource\", " + " \"granularity\": \"day\", " @@ -103,48 +114,67 @@ + " ], " + " \"intervals\": [ \"2012-01-01T00:00:00.000-08:00/2012-01-03T00:00:00.000-08:00\" ]" + " }"; - private static final String GROUP_BY_QUERY_SPLIT = + private static final String + GROUP_BY_QUERY_SPLIT = "[HiveDruidSplit{{\"queryType\":\"groupBy\"," + "\"dataSource\":{\"type\":\"table\",\"name\":\"sample_datasource\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2012-01-01T08:00:00.000Z/2012-01-03T08:00:00.000Z\"]}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2012-01-01T08:00:00" + + ".000Z/2012-01-03T08:00:00.000Z\"]}," + "\"virtualColumns\":[]," + "\"filter\":null," + "\"granularity\":\"DAY\"," - + "\"dimensions\":[{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"country\",\"outputName\":\"country\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"device\",\"outputName\":\"device\",\"outputType\":\"STRING\"}]," - + "\"aggregations\":[{\"type\":\"longSum\",\"name\":\"total_usage\",\"fieldName\":\"user_count\",\"expression\":null}," + + "\"dimensions\":[{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"country\",\"outputName\":\"country\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"device\",\"outputName\":\"device\"," + + "\"outputType\":\"STRING\"}]," + + "\"aggregations\":[{\"type\":\"longSum\",\"name\":\"total_usage\",\"fieldName\":\"user_count\"," + + "\"expression\":null}," + "{\"type\":\"doubleSum\",\"name\":\"data_transfer\",\"fieldName\":\"data_transfer\",\"expression\":null}]," + "\"postAggregations\":[]," + "\"having\":null," - + "\"limitSpec\":{\"type\":\"default\",\"columns\":[{\"dimension\":\"country\",\"direction\":\"ascending\",\"dimensionOrder\":{\"type\":\"lexicographic\"}}," - + "{\"dimension\":\"data_transfer\",\"direction\":\"ascending\",\"dimensionOrder\":{\"type\":\"lexicographic\"}}],\"limit\":5000}," + + "\"limitSpec\":{\"type\":\"default\",\"columns\":[{\"dimension\":\"country\",\"direction\":\"ascending\"," + + "\"dimensionOrder\":{\"type\":\"lexicographic\"}}," + + "{\"dimension\":\"data_transfer\",\"direction\":\"ascending\"," + + "\"dimensionOrder\":{\"type\":\"lexicographic\"}}],\"limit\":5000}," + "\"context\":{\"queryId\":\"\"}," + "\"descending\":false}, [localhost:8082]}]"; - private static final String SELECT_QUERY = + private static final String + SELECT_QUERY = "{ \"queryType\": \"select\", " + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " - + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], " + + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\"," + + "\"newpage\",\"user\"], " + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " + " \"granularity\": \"all\", " + " \"intervals\": [ \"2013-01-01T00:00:00.000-08:00/2013-01-02T00:00:00.000-08:00\" ], " + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5}, " + " \"context\":{\"druid.query.fetch\":true}}"; - private static final String SELECT_QUERY_SPLIT = + private static final String + SELECT_QUERY_SPLIT = "[HiveDruidSplit{{\"queryType\":\"select\"," + "\"dataSource\":{\"type\":\"table\",\"name\":\"wikipedia\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2013-01-01T08:00:00.000Z/2013-01-02T08:00:00.000Z\"]}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2013-01-01T08:00:00" + + ".000Z/2013-01-02T08:00:00.000Z\"]}," + "\"descending\":false," + "\"filter\":null," + "\"granularity\":{\"type\":\"all\"}," - + "\"dimensions\":[{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"robot\",\"outputName\":\"robot\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"namespace\",\"outputName\":\"namespace\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"anonymous\",\"outputName\":\"anonymous\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"unpatrolled\",\"outputName\":\"unpatrolled\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"page\",\"outputName\":\"page\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"language\",\"outputName\":\"language\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"newpage\",\"outputName\":\"newpage\",\"outputType\":\"STRING\"}," - + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"user\",\"outputName\":\"user\",\"outputType\":\"STRING\"}]," + + "\"dimensions\":[{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"robot\",\"outputName\":\"robot\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"namespace\",\"outputName\":\"namespace\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"anonymous\",\"outputName\":\"anonymous\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"unpatrolled\",\"outputName\":\"unpatrolled\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"page\",\"outputName\":\"page\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"language\",\"outputName\":\"language\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"newpage\",\"outputName\":\"newpage\"," + + "\"outputType\":\"STRING\"}," + + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"user\",\"outputName\":\"user\"," + + "\"outputType\":\"STRING\"}]," + "\"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"]," + "\"virtualColumns\":[]," + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":5,\"fromNext\":false}," @@ -153,8 +183,7 @@ public void testTimeZone() throws Exception { DruidQueryBasedInputFormat input = new DruidQueryBasedInputFormat(); - Method method1 = DruidQueryBasedInputFormat.class.getDeclaredMethod( - "getInputSplits", Configuration.class); + Method method1 = DruidQueryBasedInputFormat.class.getDeclaredMethod("getInputSplits", Configuration.class); method1.setAccessible(true); // Create, initialize, and test @@ -175,8 +204,7 @@ public void testTimeZone() throws Exception { assertEquals(SELECT_QUERY_SPLIT, Arrays.toString(resultSplits)); } - private static Configuration createPropertiesQuery(String dataSource, String queryType, - String jsonQuery) { + private static Configuration createPropertiesQuery(String dataSource, String queryType, String jsonQuery) { Configuration conf = new Configuration(); // Set the configuration parameters conf.set(FileInputFormat.INPUT_DIR, "/my/dir"); diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/io/TestHiveDruidSplit.java druid-handler/src/test/org/apache/hadoop/hive/druid/io/TestHiveDruidSplit.java index 234c783d25..be3ff6fee1 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/io/TestHiveDruidSplit.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/io/TestHiveDruidSplit.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -29,10 +29,14 @@ import java.io.DataOutputStream; import java.io.IOException; +/** + * Test Class for input splits. + */ public class TestHiveDruidSplit { - @Test - public void testSerDeser() throws IOException { - HiveDruidSplit hiveDruidSplit = new HiveDruidSplit("query string", new Path("test-path"), new String []{"host:8080", "host2:8090"}); + @Test public void testSerDeser() throws IOException { + HiveDruidSplit + hiveDruidSplit = + new HiveDruidSplit("query string", new Path("test-path"), new String[] {"host:8080", "host2:8090"}); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutput dataOutput = new DataOutputStream(byteArrayOutputStream); hiveDruidSplit.write(dataOutput); @@ -40,7 +44,6 @@ public void testSerDeser() throws IOException { HiveDruidSplit actualHiveDruidSplit = new HiveDruidSplit(); actualHiveDruidSplit.readFields(new DataInputStream(byteArrayInputStream)); Assert.assertEquals(actualHiveDruidSplit.getDruidQuery(), "query string"); - Assert.assertArrayEquals(actualHiveDruidSplit.getLocations(), new String []{"host:8080", "host2:8090"}); + Assert.assertArrayEquals(actualHiveDruidSplit.getLocations(), new String[] {"host:8080", "host2:8090"}); } - -} \ No newline at end of file +} diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/io/package-info.java druid-handler/src/test/org/apache/hadoop/hive/druid/io/package-info.java new file mode 100644 index 0000000000..3aab94b758 --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/druid/io/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info. + */ + +package org.apache.hadoop.hive.druid.io; diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/package-info.java druid-handler/src/test/org/apache/hadoop/hive/druid/package-info.java new file mode 100644 index 0000000000..c644141068 --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/druid/package-info.java @@ -0,0 +1,22 @@ +/* + * 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 info. + */ +package org.apache.hadoop.hive.druid; 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 e27f8cf0e8..74bceec169 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 @@ -122,7 +122,8 @@ + " \"name\": \"sample_divide\", " + " \"fn\": \"/\", " + " \"fields\": [ " - + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" }, " + + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" " + + "}, " + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" } " + " ] " + " } " @@ -151,13 +152,17 @@ private byte[] scanQueryResults; // Timeseries query results as records - private static final Object[][] - TIMESERIES_QUERY_RESULTS_RECORDS = - new Object[][] { new Object[] { + 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 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) } }; + 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"; @@ -262,23 +267,37 @@ + " }]"; // TopN query results as records - private static final Object[][] - TOPN_QUERY_RESULTS_RECORDS = - new Object[][] { new Object[] { + 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 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 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 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) } }; + 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"; @@ -406,7 +425,7 @@ + " } " + " }]"; - private final static String + private static final String GB_MONTH_EXTRACTIONS = "{\"queryType\":\"groupBy\",\"dataSource\":\"sample_datasource\"," + "\"granularity\":\"all\"," @@ -416,33 +435,41 @@ + "\"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[] { + 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 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[] { + 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 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 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[] { + 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 IntWritable(1), + new LongWritable(200) }, + new Object[]{ new TimestampLocalTZWritable(new TimestampTZ(Instant.ofEpochMilli(1325376012000L).atZone(ZoneOffset.UTC))), - new IntWritable(1), new LongWritable(400) } }; + 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"; @@ -463,7 +490,8 @@ SELECT_QUERY = "{ \"queryType\": \"select\", " + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " - + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], " + + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\"," + + "\"newpage\",\"user\"], " + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " + " \"granularity\": \"all\", " + " \"intervals\": [ \"2013-01-01/2013-01-02\" ], " @@ -478,7 +506,8 @@ + " \"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\", " + + " \"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\", " @@ -497,7 +526,8 @@ + " \"deleted\" : 0.0 " + " } " + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"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\", " @@ -516,7 +546,8 @@ + " \"deleted\" : 0.0 " + " } " + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"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\", " @@ -535,7 +566,8 @@ + " \"deleted\" : 0.0 " + " } " + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"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\", " @@ -554,7 +586,8 @@ + " \"deleted\" : 0.0 " + " } " + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"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\", " @@ -580,37 +613,92 @@ "__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[] { + "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 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 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 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 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) } }; + 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\"], " + + " \"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\"," @@ -624,11 +712,16 @@ + "\"columns\":[\"__time\",\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\"," + "\"newpage\",\"user\",\"count\",\"added\",\"delta\",\"variation\",\"deleted\"]," + "\"events\":[" - + "[\"2013-01-01T00:00:00.000Z\", 1,\"article\",\"0\",\"0\",\"11._korpus_(NOVJ)\",\"sl\",\"0\",\"EmausBot\",1.0,39.0,39.0,39.0,0.0]," - + "[\"2013-01-01T00:00:00.000Z\", 0,\"article\",\"0\",\"0\",\"112_U.S._580\",\"en\",\"1\",\"MZMcBride\",1.0,70.0,70.0,70.0,0.0]," - + "[\"2013-01-01T00:00:12.000Z\", 0,\"article\",\"0\",\"0\",\"113_U.S._243\",\"en\",\"1\",\"MZMcBride\",1.0,77.0,77.0,77.0,0.0]," - + "[\"2013-01-01T00:00:12.000Z\", 0,\"article\",\"0\",\"0\",\"113_U.S._73\",\"en\",\"1\",\"MZMcBride\",1.0,70.0,70.0,70.0,0.0]," - + "[\"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]" + + "[\"2013-01-01T00:00:00.000Z\", 1,\"article\",\"0\",\"0\",\"11._korpus_(NOVJ)\",\"sl\",\"0\"," + + "\"EmausBot\",1.0,39.0,39.0,39.0,0.0]," + + "[\"2013-01-01T00:00:00.000Z\", 0,\"article\",\"0\",\"0\",\"112_U.S._580\",\"en\",\"1\",\"MZMcBride\",1" + + ".0,70.0,70.0,70.0,0.0]," + + "[\"2013-01-01T00:00:12.000Z\", 0,\"article\",\"0\",\"0\",\"113_U.S._243\",\"en\",\"1\",\"MZMcBride\",1" + + ".0,77.0,77.0,77.0,0.0]," + + "[\"2013-01-01T00:00:12.000Z\", 0,\"article\",\"0\",\"0\",\"113_U.S._73\",\"en\",\"1\",\"MZMcBride\",1.0," + + "70.0,70.0,70.0,0.0]," + + "[\"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 { @@ -774,14 +867,10 @@ private static Properties createPropertiesQuery(String dataSource, when(httpClient.go(anyObject(), any(HttpResponseHandler.class))).thenReturn(futureResult); 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"}); assert reader != null; - reader.initialize(split, - new Configuration(), - DruidStorageHandlerUtils.JSON_MAPPER, - DruidStorageHandlerUtils.SMILE_MAPPER, - httpClient); + reader.initialize(split, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, httpClient); StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector(); List fieldRefs = oi.getAllStructFieldRefs(); @@ -807,11 +896,7 @@ private static Properties createPropertiesQuery(String dataSource, when(httpClient.go(anyObject(), any(HttpResponseHandler.class))).thenReturn(futureResult); reader = DruidQueryBasedInputFormat.getDruidQueryReader(queryType); assert reader != null; - reader.initialize(split, - new Configuration(), - DruidStorageHandlerUtils.JSON_MAPPER, - DruidStorageHandlerUtils.SMILE_MAPPER, - httpClient); + reader.initialize(split, DruidStorageHandlerUtils.JSON_MAPPER, DruidStorageHandlerUtils.SMILE_MAPPER, httpClient); pos = 0; while (reader.nextKeyValue()) { @@ -832,16 +917,20 @@ private static Properties createPropertiesQuery(String dataSource, 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 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) @@ -880,13 +969,20 @@ private static Properties createPropertiesQuery(String dataSource, // 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 @@ -931,14 +1027,17 @@ private static void serializeObject(Properties properties, } } - 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 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) diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/serde/package-info.java druid-handler/src/test/org/apache/hadoop/hive/druid/serde/package-info.java new file mode 100644 index 0000000000..c966779f51 --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/druid/serde/package-info.java @@ -0,0 +1,22 @@ +/* + * 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 info. + */ +package org.apache.hadoop.hive.druid.serde; diff --git druid-handler/src/test/org/apache/hadoop/hive/package-info.java druid-handler/src/test/org/apache/hadoop/hive/package-info.java new file mode 100644 index 0000000000..5a8e5a7fc4 --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info. + */ + +package org.apache.hadoop.hive; 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 63efdc4d8f..ed6e430c8a 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 @@ -140,7 +140,7 @@ dataSchema = new DataSchema(dataSourceName, parserMap, - new AggregatorFactory[] { new LongSumAggregatorFactory("visited_sum", "visited_sum"), + new AggregatorFactory[] {new LongSumAggregatorFactory("visited_sum", "visited_sum"), new HyperUniquesAggregatorFactory("unique_hosts", "unique_hosts") }, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)), null, @@ -184,7 +184,9 @@ 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(DruidConstants.DEFAULT_TIMESTAMP_COLUMN))) + Granularities + .DAY + .bucketStart(new DateTime((long) input.get(DruidConstants.DEFAULT_TIMESTAMP_COLUMN))) .getMillis()) .build())) .collect(Collectors.toList()); @@ -240,7 +242,15 @@ private void verifyRows(List> expectedRows, List Alias: hdfs://### HDFS PATH ### [languages] @@ -336,7 +336,7 @@ STAGE PLANS: tag: 1 value expressions: user (type: string) auto parallelism: true - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: no inputs Path -> Alias: hdfs://### HDFS PATH ### [druid_kafka_test] diff --git ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out index b07ed52d06..270a51c684 100644 --- ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out +++ ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out @@ -10,7 +10,7 @@ SELECT count(*) FROM druid_table_alltypesorc WHERE character_length(CAST(ctinyin POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -837 +895 PREHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE character_length(CAST(ctinyint AS STRING)) > 1 AND char_length(CAST(ctinyint AS STRING)) < 10 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -19,7 +19,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE character_le POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -4838 +4953 PREHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE power(cfloat, 2) * pow(csmallint, 3) > 1 AND SQRT(ABS(ctinyint)) > 3 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -28,7 +28,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc WHERE power(cfloat POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -837 +895 PREHOOK: query: SELECT SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint) FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 1000 OR ln(cdouble) / log10(10) > 0 AND COS(cint) > 0 OR SIN(cdouble) > 1 PREHOOK: type: QUERY @@ -39,7 +39,7 @@ FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -19884.646918177605 27373419 14472 8.51628242804E11 851620413654 68151649880 +20184.307997584343 7782252 14776 9.04717176052E11 904709411083 68151649880 PREHOOK: query: SELECT SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint) FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 1000 PREHOOK: type: QUERY @@ -50,7 +50,7 @@ FROM druid_table_alltypesorc WHERE ceil(cfloat) > 0 AND floor(cdouble) * 2 < 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -54430.269943237305 -3740445 51268 1.31919188502E11 131922984948 92160895030 +56644.54301452637 -4586617 53410 3.00451922196E11 300456560786 89357678084 PREHOOK: query: SELECT SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint) FROM druid_table_alltypesorc WHERE ln(cdouble) / log10(10) > 0 AND COS(cint) > 0 OR SIN(cdouble) > 1 PREHOOK: type: QUERY @@ -61,7 +61,7 @@ FROM druid_table_alltypesorc WHERE ln(cdouble) / log10(10) > 0 AND COS(cint) > POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### --2389.517022252083 27640645 -5707 7.19705549994E11 719697428706 13774723379 +-3212.129014134407 8379141 -6468 7.14599285388E11 714590903722 13774723379 PREHOOK: query: SELECT SUM(cfloat + 1), CAST(SUM(cdouble + ctinyint) AS INTEGER), SUM(ctinyint) + 1 , CAST(SUM(csmallint) + SUM(cint) AS DOUBLE), SUM(cint), SUM(cbigint) FROM druid_table_alltypesorc WHERE SIN(cdouble) > 1 PREHOOK: type: QUERY @@ -226,7 +226,7 @@ STAGE PLANS: tag: -1 value expressions: _col0 (type: bigint) auto parallelism: false - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: no inputs Path -> Alias: hdfs://### HDFS PATH ### [druid_table_alltypesorc] @@ -249,7 +249,7 @@ STAGE PLANS: druid.datasource default.druid_table_alltypesorc druid.fieldNames vc druid.fieldTypes int - druid.query.granularity MINUTE + druid.query.granularity SECOND druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"0","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList","limit":1} druid.query.type scan druid.segment.granularity HOUR @@ -282,7 +282,7 @@ STAGE PLANS: druid.datasource default.druid_table_alltypesorc druid.fieldNames vc druid.fieldTypes int - druid.query.granularity MINUTE + druid.query.granularity SECOND druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"0","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList","limit":1} druid.query.type scan druid.segment.granularity HOUR @@ -304,7 +304,7 @@ STAGE PLANS: Truncated Path -> Alias: /druid_table_alltypesorc [druid_table_alltypesorc] Reducer 2 - Execution mode: llap + Execution mode: vectorized, llap Needs Tagging: false Reduce Operator Tree: Group By Operator @@ -355,16 +355,16 @@ WHERE (`__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00') POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_alltypesorc GROUP BY `__time`, `cstring1` PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -688,10 +688,10 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) @@ -751,10 +751,10 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: double) - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0), sum(VALUE._col1) @@ -814,10 +814,10 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 216 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: double), _col2 (type: int) - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0), sum(VALUE._col1), min(VALUE._col2) @@ -838,6 +838,71 @@ STAGE PLANS: Processor Tree: ListSink +PREHOOK: query: EXPLAIN select count(*) from (select `__time` from druid_table_alltypesorc limit 1025) as src +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: EXPLAIN select count(*) from (select `__time` from druid_table_alltypesorc limit 1025) as src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + properties: + druid.fieldNames vc + druid.fieldTypes int + druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"0","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList","limit":1025} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Select Operator + Statistics: Num rows: 9173 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + PREHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -846,7 +911,7 @@ POSTHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_ POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6078 2.7308662809692383E7 +6078 2.730866281640625E7 PREHOOK: query: select count(distinct cstring2), sum(2 * cdouble) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -855,7 +920,7 @@ POSTHOOK: query: select count(distinct cstring2), sum(2 * cdouble) FROM druid_ta POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6078 5.4617325619384766E7 +6078 5.46173256328125E7 PREHOOK: query: select count(DISTINCT cstring2) FROM druid_table_alltypesorc PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -865,6 +930,15 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### 6078 +PREHOOK: query: select count(DISTINCT cstring1) FROM druid_table_alltypesorc +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(DISTINCT cstring1) FROM druid_table_alltypesorc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +3025 PREHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_alltypesorc PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -873,7 +947,7 @@ POSTHOOK: query: select count(DISTINCT cstring2), sum(cdouble) FROM druid_table_ POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6078 2.7308662809692383E7 +6078 2.730866281640625E7 PREHOOK: query: select count(distinct cstring2 || '_'|| cstring1), sum(cdouble), min(cint) FROM druid_table_alltypesorc PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -882,7 +956,233 @@ POSTHOOK: query: select count(distinct cstring2 || '_'|| cstring1), sum(cdouble) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6095 2.7308662809692383E7 -1073279343 +6095 2.730866281640625E7 -1073279343 +PREHOOK: query: select count(*) from (select `__time` from druid_table_alltypesorc limit 1025) as src +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from (select `__time` from druid_table_alltypesorc limit 1025) as src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +1025 +PREHOOK: query: select count(*) from (select `__time` from druid_table_alltypesorc limit 200000) as src +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from (select `__time` from druid_table_alltypesorc limit 200000) as src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +6226 +PREHOOK: query: select count(`__time`) from (select `__time` from druid_table_alltypesorc limit 200) as src +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(`__time`) from (select `__time` from druid_table_alltypesorc limit 200) as src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +200 +PREHOOK: query: select count(distinct `__time`) from druid_table_alltypesorc +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(distinct `__time`) from druid_table_alltypesorc +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +35 +PREHOOK: query: explain select `timets` from (select `__time` as timets from druid_table_alltypesorc order by timets limit 10) as src order by `timets` +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select `timets` from (select `__time` as timets from druid_table_alltypesorc order by timets limit 10) as src order by `timets` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + properties: + druid.fieldNames vc + druid.fieldTypes timestamp with local time zone + druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"\"__time\"","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList"} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: vc (type: timestamp with local time zone) + outputColumnNames: _col0 + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: timestamp with local time zone) + sort order: + + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Execution mode: llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: timestamp with local time zone) + outputColumnNames: _col0 + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: timestamp with local time zone) + sort order: + + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: timestamp with local time zone) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain select `timets` from (select cast(`__time` as timestamp ) as timets from druid_table_alltypesorc order by timets limit 10) as src order by `timets` +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: explain select `timets` from (select cast(`__time` as timestamp ) as timets from druid_table_alltypesorc order by timets limit 10) as src order by `timets` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Reducer 2 <- Map 1 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: druid_table_alltypesorc + properties: + druid.fieldNames vc + druid.fieldTypes timestamp + druid.query.json {"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"virtualColumns":[{"type":"expression","name":"vc","expression":"timestamp_parse(timestamp_format(\"__time\",'yyyy-MM-dd\\u0027T\\u0027HH:mm:ss.SSS\\u0027Z\\u0027','US/Pacific'),'','UTC')","outputType":"LONG"}],"columns":["vc"],"resultFormat":"compactedList"} + druid.query.type scan + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: vc (type: timestamp) + outputColumnNames: _col0 + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: timestamp) + sort order: + + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Execution mode: vectorized, llap + LLAP IO: no inputs + Reducer 2 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: timestamp) + outputColumnNames: _col0 + Statistics: Num rows: 9173 Data size: 348640 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: timestamp) + sort order: + + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: timestamp) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 10 Data size: 380 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select `timets_with_tz` from (select `__time` as timets_with_tz from druid_table_alltypesorc order by timets_with_tz limit 10) as src order by `timets_with_tz` +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `timets_with_tz` from (select `__time` as timets_with_tz from druid_table_alltypesorc order by timets_with_tz limit 10) as src order by `timets_with_tz` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +PREHOOK: query: select `timets` from (select cast(`__time` as timestamp ) as timets from druid_table_alltypesorc order by timets limit 10) as src order by `timets` +PREHOOK: type: QUERY +PREHOOK: Input: default@druid_table_alltypesorc +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `timets` from (select cast(`__time` as timestamp ) as timets from druid_table_alltypesorc order by timets limit 10) as src order by `timets` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@druid_table_alltypesorc +POSTHOOK: Output: hdfs://### HDFS PATH ### +1969-12-31 15:59:30 +1969-12-31 15:59:30 +1969-12-31 15:59:30 +1969-12-31 15:59:30 +1969-12-31 15:59:43 +1969-12-31 15:59:43 +1969-12-31 15:59:43 +1969-12-31 15:59:43 +1969-12-31 15:59:43 +1969-12-31 15:59:43 PREHOOK: query: explain select unix_timestamp(from_unixtime(1396681200)) from druid_table_alltypesorc limit 1 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -956,7 +1256,7 @@ POSTHOOK: query: select unix_timestamp(`__time`) from druid_table_alltypesorc li POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### --60 +-30 PREHOOK: query: explain select FROM_UNIXTIME(UNIX_TIMESTAMP(CAST(`__time` as timestamp ),'yyyy-MM-dd HH:mm:ss' ),'yyyy-MM-dd HH:mm:ss') from druid_table_alltypesorc GROUP BY FROM_UNIXTIME(UNIX_TIMESTAMP(CAST(`__time` as timestamp ),'yyyy-MM-dd HH:mm:ss' ),'yyyy-MM-dd HH:mm:ss') @@ -1001,8 +1301,41 @@ GROUP BY FROM_UNIXTIME(UNIX_TIMESTAMP(CAST(`__time` as timestamp ),'yyyy-MM-dd H POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00 +1969-12-31 15:59:30 +1969-12-31 15:59:43 +1969-12-31 15:59:44 +1969-12-31 15:59:45 +1969-12-31 15:59:46 +1969-12-31 15:59:47 +1969-12-31 15:59:48 +1969-12-31 15:59:49 +1969-12-31 15:59:50 +1969-12-31 15:59:51 +1969-12-31 15:59:52 +1969-12-31 15:59:53 +1969-12-31 15:59:54 +1969-12-31 15:59:55 +1969-12-31 15:59:56 +1969-12-31 15:59:57 +1969-12-31 15:59:58 1969-12-31 16:00:00 +1969-12-31 16:00:01 +1969-12-31 16:00:02 +1969-12-31 16:00:03 +1969-12-31 16:00:04 +1969-12-31 16:00:05 +1969-12-31 16:00:06 +1969-12-31 16:00:07 +1969-12-31 16:00:08 +1969-12-31 16:00:09 +1969-12-31 16:00:10 +1969-12-31 16:00:11 +1969-12-31 16:00:12 +1969-12-31 16:00:13 +1969-12-31 16:00:14 +1969-12-31 16:00:15 +1969-12-31 16:00:16 +1969-12-31 16:00:31 PREHOOK: query: explain select TRUNC(cast(`__time` as timestamp), 'YY') from druid_table_alltypesorc GROUP BY TRUNC(cast(`__time` as timestamp), 'YY') PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -1244,7 +1577,7 @@ GROUP BY CAST(TRUNC(CAST(`druid_table_alias`.`__time` AS TIMESTAMP),'MM') AS DAT POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -3.832948323764436E14 1969-12-01 +1.919174797621384E14 1969-12-01 PREHOOK: query: explain SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_1, DATE_SUB(cast(`__time` as date), CAST((cdouble / 1000) AS INT)) as date_2 from druid_table_alltypesorc order by date_1, date_2 limit 3 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -1284,10 +1617,10 @@ STAGE PLANS: sort order: ++ Statistics: Num rows: 9173 Data size: 976192 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: llap + Execution mode: vectorized, llap LLAP IO: no inputs Reducer 2 - Execution mode: llap + Execution mode: vectorized, llap Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: date), KEY.reducesinkkey1 (type: date) @@ -1318,9 +1651,9 @@ POSTHOOK: query: SELECT DATE_ADD(cast(`__time` as date), CAST((cdouble / 1000) A POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-02-26 1970-11-04 -1969-03-19 1970-10-14 -1969-11-13 1970-02-17 +1969-08-29 1970-05-04 +1969-09-21 1970-04-11 +1969-10-02 1970-03-31 PREHOOK: query: -- Boolean Values @@ -1363,8 +1696,8 @@ POSTHOOK: query: SELECT cboolean2, count(*) from druid_table_alltypesorc GROUP B POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -false 3148 -true 2957 +false 3268 +true 2958 PREHOOK: query: -- Expected results of this query are wrong due to https://issues.apache.org/jira/browse/CALCITE-2319 -- It should get fixed once we upgrade calcite SELECT ctinyint > 2, count(*) from druid_table_alltypesorc GROUP BY ctinyint > 2 @@ -1377,8 +1710,8 @@ POSTHOOK: query: -- Expected results of this query are wrong due to https://issu POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -false 2653 -true 3452 +false 2718 +true 3508 PREHOOK: query: EXPLAIN SELECT ctinyint > 2, count(*) from druid_table_alltypesorc GROUP BY ctinyint > 2 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -1443,7 +1776,7 @@ POSTHOOK: query: SELECT sum(cfloat) FROM druid_table_alltypesorc WHERE cstring1 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### --39590.24724686146 +-39590.24699509144 PREHOOK: query: EXPLAIN SELECT sum(cfloat) FROM druid_table_alltypesorc WHERE cstring1 != 'en' group by 1.011, 3.40 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -1480,7 +1813,7 @@ POSTHOOK: query: SELECT sum(cfloat) FROM druid_table_alltypesorc WHERE cstring1 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### --39590.24724686146 +-39590.24699509144 PREHOOK: query: EXPLAIN SELECT sum(cint) FROM druid_table_alltypesorc WHERE cfloat= 0.011 group by cfloat PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc diff --git ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out index 4ea95f6930..de49ef648a 100644 --- ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out +++ ql/src/test/results/clientpositive/druid/druidmini_extractTime.q.out @@ -11,6 +11,40 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### 0 +1 +10 +11 +12 +13 +14 +15 +16 +2 +3 +30 +31 +4 +43 +44 +45 +46 +47 +48 +49 +5 +50 +51 +52 +53 +54 +55 +56 +57 +58 +6 +7 +8 +9 PREHOOK: query: EXPLAIN SELECT EXTRACT(SECOND from `__time`) FROM druid_table_alltypesorc WHERE character_length(CAST(ctinyint AS STRING)) > 1 AND char_length(CAST(ctinyint AS STRING)) < 10 AND power(cfloat, 2) * pow(csmallint, 3) > 1 AND SQRT(ABS(ctinyint)) > 3 GROUP BY EXTRACT(SECOND from `__time`) @@ -725,8 +759,11 @@ POSTHOOK: query: SELECT CAST(`__time` AS TIMESTAMP) AS `x_time`, SUM(cfloat) FR POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00 -4532.570196151733 -1969-12-31 16:00:00 -35057.677050709724 +1969-12-31 15:59:30 49.0 +1969-12-31 15:59:43 41.610992431640625 +1969-12-31 15:59:44 -204.72699165344238 +1969-12-31 15:59:45 -588.0 +1969-12-31 15:59:46 56.0 PREHOOK: query: explain SELECT CAST(`__time` AS DATE) AS `x_date`, SUM(cfloat) FROM druid_table_alltypesorc GROUP BY CAST(`__time` AS DATE) ORDER BY `x_date` LIMIT 5 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -763,7 +800,7 @@ POSTHOOK: query: SELECT CAST(`__time` AS DATE) AS `x_date`, SUM(cfloat) FROM dr POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 -39590.24724686146 +1969-12-31 -39590.24699509144 PREHOOK: query: SELECT CAST(`__time` AS DATE) AS `x_date` FROM druid_table_alltypesorc ORDER BY `x_date` LIMIT 5 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc diff --git ql/src/test/results/clientpositive/druid/druidmini_floorTime.q.out ql/src/test/results/clientpositive/druid/druidmini_floorTime.q.out index 8d9382443e..902c8f73a6 100644 --- ql/src/test/results/clientpositive/druid/druidmini_floorTime.q.out +++ ql/src/test/results/clientpositive/druid/druidmini_floorTime.q.out @@ -10,8 +10,41 @@ AND power(cfloat, 2) * pow(csmallint, 3) > 1 AND SQRT(ABS(ctinyint)) > 3 GROUP B POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:44.0 US/Pacific +1969-12-31 15:59:45.0 US/Pacific +1969-12-31 15:59:46.0 US/Pacific +1969-12-31 15:59:47.0 US/Pacific +1969-12-31 15:59:48.0 US/Pacific +1969-12-31 15:59:49.0 US/Pacific +1969-12-31 15:59:50.0 US/Pacific +1969-12-31 15:59:51.0 US/Pacific +1969-12-31 15:59:52.0 US/Pacific +1969-12-31 15:59:53.0 US/Pacific +1969-12-31 15:59:54.0 US/Pacific +1969-12-31 15:59:55.0 US/Pacific +1969-12-31 15:59:56.0 US/Pacific +1969-12-31 15:59:57.0 US/Pacific +1969-12-31 15:59:58.0 US/Pacific 1969-12-31 16:00:00.0 US/Pacific +1969-12-31 16:00:01.0 US/Pacific +1969-12-31 16:00:02.0 US/Pacific +1969-12-31 16:00:03.0 US/Pacific +1969-12-31 16:00:04.0 US/Pacific +1969-12-31 16:00:05.0 US/Pacific +1969-12-31 16:00:06.0 US/Pacific +1969-12-31 16:00:07.0 US/Pacific +1969-12-31 16:00:08.0 US/Pacific +1969-12-31 16:00:09.0 US/Pacific +1969-12-31 16:00:10.0 US/Pacific +1969-12-31 16:00:11.0 US/Pacific +1969-12-31 16:00:12.0 US/Pacific +1969-12-31 16:00:13.0 US/Pacific +1969-12-31 16:00:14.0 US/Pacific +1969-12-31 16:00:15.0 US/Pacific +1969-12-31 16:00:16.0 US/Pacific +1969-12-31 16:00:31.0 US/Pacific PREHOOK: query: EXPLAIN SELECT floor(`__time` to SECOND) FROM druid_table_alltypesorc WHERE character_length(CAST(ctinyint AS STRING)) > 1 AND char_length(CAST(ctinyint AS STRING)) < 10 AND power(cfloat, 2) * pow(csmallint, 3) > 1 AND SQRT(ABS(ctinyint)) > 3 GROUP BY floor(`__time` to SECOND) diff --git ql/src/test/results/clientpositive/druid/druidmini_test1.q.out ql/src/test/results/clientpositive/druid/druidmini_test1.q.out index 7aa1763161..513f35cf0c 100644 --- ql/src/test/results/clientpositive/druid/druidmini_test1.q.out +++ ql/src/test/results/clientpositive/druid/druidmini_test1.q.out @@ -34,7 +34,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6105 +6226 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), SUM(cfloat), SUM(cdouble), SUM(ctinyint), SUM(csmallint),SUM(cint), SUM(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -75,7 +75,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -39590.24724686146 2.7308662809692383E7 -39967 7781089 1408069801800 10992545287 +1969-01-01 00:00:00.0 US/Pacific -39590.24699509144 2.730866281640625E7 -39967 7781089 1408069801800 10992545287 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), MIN(cfloat), MIN(cdouble), MIN(ctinyint), MIN(csmallint),MIN(cint), MIN(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -116,7 +116,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -1790.7781 -308691.84375 2 14255 -1073279343 -8577981133 +1969-01-01 00:00:00.0 US/Pacific -525.0 -124870.0 -13 6202 -1073279343 -3486702173 PREHOOK: query: EXPLAIN SELECT floor_year(`__time`), MAX(cfloat), MAX(cdouble), MAX(ctinyint), MAX(csmallint),MAX(cint), MAX(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -157,7 +157,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific 769.16394 1.9565518E7 -45 -8101 1276572707 4923772860 +1969-01-01 00:00:00.0 US/Pacific 479.0 9806671.0 -33 -17520 523556254 3423398920 PREHOOK: query: EXPLAIN SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -194,8 +194,8 @@ POSTHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1cGVWH7n1QU -596096.6875 -821UdmGbkEf4j -14161.827026367188 +1cGVWH7n1QU -596096.681640625 +821UdmGbkEf4j -14161.826171875 00iT08 0.0 02v8WnLuYDos3Cq 0.0 yv1js 0.0 @@ -240,7 +240,7 @@ POSTHOOK: query: SELECT cstring2, MAX(cdouble) FROM druid_table_alltypesorc GROU POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -NULL 1.9565518E7 +NULL 9806671.0 0034fkcXMQI3 15601.0 004J8y 0.0 00GNm -200.0 @@ -329,16 +329,16 @@ FROM druid_table_alltypesorc ORDER BY `__time` ASC LIMIT 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN SELECT `__time` FROM druid_table_alltypesorc @@ -422,16 +422,16 @@ WHERE `__time` < '1970-03-01 00:00:00' ORDER BY `__time` ASC LIMIT 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN SELECT `__time` FROM druid_table_alltypesorc @@ -515,16 +515,16 @@ WHERE `__time` >= '1968-01-01 00:00:00' AND `__time` <= '1970-03-01 00:00:00' OR POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN SELECT `__time` FROM druid_table_alltypesorc @@ -612,16 +612,16 @@ WHERE `__time` >= '1968-01-01 00:00:00' AND `__time` <= '1970-03-01 00:00:00' POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN SELECT `__time` FROM druid_table_alltypesorc @@ -705,16 +705,16 @@ WHERE `__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00' ORDER BY POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN SELECT `__time` FROM druid_table_alltypesorc @@ -802,16 +802,16 @@ WHERE (`__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00') POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: explain select (cstring1 is null ) AS is_null, (cint is not null ) as isnotnull FROM druid_table_alltypesorc PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc diff --git ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out index 9c412d97dd..d873ac01aa 100644 --- ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out +++ ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out @@ -6,7 +6,7 @@ POSTHOOK: query: SELECT count(*) FROM druid_table_alltypesorc POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -6105 +6226 PREHOOK: query: SELECT floor_year(`__time`), SUM(cfloat), SUM(cdouble), SUM(ctinyint), SUM(csmallint),SUM(cint), SUM(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -17,7 +17,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -39590.24724686146 2.7308662809692383E7 -39967 7781089 1408069801800 10992545287 +1969-01-01 00:00:00.0 US/Pacific -39590.24699509144 2.730866281640625E7 -39967 7781089 1408069801800 10992545287 PREHOOK: query: SELECT floor_year(`__time`), MIN(cfloat), MIN(cdouble), MIN(ctinyint), MIN(csmallint),MIN(cint), MIN(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -28,7 +28,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific -1790.7781 -308691.84375 2 14255 -1073279343 -8577981133 +1969-01-01 00:00:00.0 US/Pacific -525.0 -124870.0 -13 6202 -1073279343 -3486702173 PREHOOK: query: SELECT floor_year(`__time`), MAX(cfloat), MAX(cdouble), MAX(ctinyint), MAX(csmallint),MAX(cint), MAX(cbigint) FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) PREHOOK: type: QUERY @@ -39,7 +39,7 @@ FROM druid_table_alltypesorc GROUP BY floor_year(`__time`) POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-01-01 00:00:00.0 US/Pacific 769.16394 1.9565518E7 -45 -8101 1276572707 4923772860 +1969-01-01 00:00:00.0 US/Pacific 479.0 9806671.0 -33 -17520 523556254 3423398920 PREHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc GROUP BY cstring1 ORDER BY s ASC LIMIT 10 PREHOOK: type: QUERY PREHOOK: Input: default@druid_table_alltypesorc @@ -48,8 +48,8 @@ POSTHOOK: query: SELECT cstring1, SUM(cdouble) as s FROM druid_table_alltypesorc POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1cGVWH7n1QU -596096.6875 -821UdmGbkEf4j -14161.827026367188 +1cGVWH7n1QU -596096.681640625 +821UdmGbkEf4j -14161.826171875 00iT08 0.0 02v8WnLuYDos3Cq 0.0 yv1js 0.0 @@ -66,7 +66,7 @@ POSTHOOK: query: SELECT cstring2, MAX(cdouble) FROM druid_table_alltypesorc GROU POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -NULL 1.9565518E7 +NULL 9806671.0 0034fkcXMQI3 15601.0 004J8y 0.0 00GNm -200.0 @@ -86,16 +86,16 @@ FROM druid_table_alltypesorc ORDER BY `__time` ASC LIMIT 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: SELECT `__time` FROM druid_table_alltypesorc WHERE `__time` < '1970-03-01 00:00:00' ORDER BY `__time` ASC LIMIT 10 @@ -108,16 +108,16 @@ WHERE `__time` < '1970-03-01 00:00:00' ORDER BY `__time` ASC LIMIT 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: SELECT `__time` FROM druid_table_alltypesorc WHERE `__time` >= '1968-01-01 00:00:00' AND `__time` <= '1970-03-01 00:00:00' ORDER BY `__time` ASC LIMIT 10 @@ -130,16 +130,16 @@ WHERE `__time` >= '1968-01-01 00:00:00' AND `__time` <= '1970-03-01 00:00:00' OR POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: SELECT `__time` FROM druid_table_alltypesorc WHERE `__time` >= '1968-01-01 00:00:00' AND `__time` <= '1970-03-01 00:00:00' @@ -154,16 +154,16 @@ WHERE `__time` >= '1968-01-01 00:00:00' AND `__time` <= '1970-03-01 00:00:00' POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: SELECT `__time` FROM druid_table_alltypesorc WHERE `__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00' ORDER BY `__time` ASC LIMIT 10 @@ -176,16 +176,16 @@ WHERE `__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00' ORDER BY POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: SELECT `__time` FROM druid_table_alltypesorc WHERE (`__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00') @@ -200,16 +200,16 @@ WHERE (`__time` BETWEEN '1968-01-01 00:00:00' AND '1970-01-01 00:00:00') POSTHOOK: type: QUERY POSTHOOK: Input: default@druid_table_alltypesorc POSTHOOK: Output: hdfs://### HDFS PATH ### -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific -1969-12-31 15:59:00.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:30.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific +1969-12-31 15:59:43.0 US/Pacific PREHOOK: query: EXPLAIN SELECT `__time` FROM druid_table_alltypesorc diff --git serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java index ef5e5a6cc7..94922dac8d 100644 --- serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java +++ serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/PrimitiveObjectInspector.java @@ -33,7 +33,7 @@ /** * The primitive types supported by Hive. */ - public static enum PrimitiveCategory { + enum PrimitiveCategory { VOID, BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, STRING, DATE, TIMESTAMP, TIMESTAMPLOCALTZ, BINARY, DECIMAL, VARCHAR, CHAR, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME, UNKNOWN diff --git serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java index 51a0aed16a..9cbe1c1ab0 100644 --- serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java +++ serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java @@ -237,7 +237,7 @@ } private static Map primitiveCategoryToJavaOI = - new EnumMap(PrimitiveCategory.class); + new EnumMap<>(PrimitiveCategory.class); static { primitiveCategoryToJavaOI.put(PrimitiveCategory.BOOLEAN, javaBooleanObjectInspector); primitiveCategoryToJavaOI.put(PrimitiveCategory.BYTE, javaByteObjectInspector); @@ -293,16 +293,16 @@ public static AbstractPrimitiveWritableObjectInspector getPrimitiveWritableObjec result = new WritableHiveCharObjectInspector((CharTypeInfo) typeInfo); break; case VARCHAR: - result = new WritableHiveVarcharObjectInspector((VarcharTypeInfo)typeInfo); + result = new WritableHiveVarcharObjectInspector((VarcharTypeInfo) typeInfo); break; case TIMESTAMPLOCALTZ: - result = new WritableTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo); + result = new WritableTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo) typeInfo); break; case DECIMAL: - result = new WritableHiveDecimalObjectInspector((DecimalTypeInfo)typeInfo); + result = new WritableHiveDecimalObjectInspector((DecimalTypeInfo) typeInfo); break; default: - throw new RuntimeException("Failed to create object inspector for " + typeInfo ); + throw new RuntimeException("Failed to create object inspector for " + typeInfo); } AbstractPrimitiveWritableObjectInspector prev = diff --git storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java index 01bd2f6e88..da2673d0cf 100644 --- storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java +++ storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java @@ -35,7 +35,7 @@ /* * The current kinds of column vectors. */ - public static enum Type { + public enum Type { NONE, // Useful when the type of column vector has not be determined yet. LONG, DOUBLE, @@ -107,7 +107,7 @@ public void reset() { /** * Sets the isRepeating flag. Recurses over structs and unions so that the * flags are set correctly. - * @param isRepeating + * @param isRepeating flag for repeating value. */ public void setRepeating(boolean isRepeating) { this.isRepeating = isRepeating; diff --git storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java index f8ed7e29e5..35234a4ec9 100644 --- storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java +++ storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java @@ -181,7 +181,7 @@ public String stringifyColumn(int columnNum) { } private void appendVectorType(StringBuilder b, ColumnVector cv) { - String colVectorType = null; + String colVectorType; if (cv instanceof LongColumnVector) { colVectorType = "LONG"; } else if (cv instanceof DoubleColumnVector) { @@ -355,8 +355,8 @@ public void reset() { * Data is not preserved. */ public void ensureSize(int rows) { - for(int i=0; i < cols.length; ++i) { - cols[i].ensureSize(rows, false); + for (ColumnVector col : cols) { + col.ensureSize(rows, false); } } }