From 8b7826db3a20b8abf35a070fa2f3cb3257d43608 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 2 May 2017 22:38:04 +0900 Subject: [PATCH 01/15] Early publishing segments in the middle of data ingestion --- .../druid/indexing/kafka/KafkaIndexTask.java | 77 ++-- .../indexing/kafka/KafkaTuningConfig.java | 7 + .../ActionBasedSegmentAllocator.java | 2 + .../druid/indexing/common/task/IndexTask.java | 336 ++++++++++-------- .../indexing/common/task/IndexTaskTest.java | 85 ++++- .../indexing/common/task/TaskSerdeTest.java | 4 +- .../indexing/overlord/TaskLifecycleTest.java | 6 +- .../indexing/RealtimeTuningConfig.java | 7 + .../realtime/appenderator/Appenderator.java | 2 + .../appenderator/AppenderatorConfig.java | 2 + .../appenderator/AppenderatorImpl.java | 22 +- .../realtime/appenderator/Appenderators.java | 6 + .../FiniteAppenderatorDriver.java | 103 +++++- .../appenderator/SegmentAllocator.java | 2 + .../appenderator/AppenderatorPlumberTest.java | 2 +- .../appenderator/AppenderatorTest.java | 42 +++ .../FiniteAppenderatorDriverTest.java | 70 +++- 17 files changed, 535 insertions(+), 240 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index eef8aac5d714..78b878456bb9 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -92,11 +92,11 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.Set; -import java.util.Collections; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; @@ -355,6 +355,39 @@ public void run() } }; + final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() + { + @Override + public boolean publishSegments(Set segments, Object commitMetadata) throws IOException + { + final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), + KafkaPartitions.class + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { + throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); + } + + final SegmentTransactionalInsertAction action; + + if (ioConfig.isUseTransaction()) { + action = new SegmentTransactionalInsertAction( + segments, + new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), + new KafkaDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); + + return toolbox.getTaskActionClient().submit(action).isSuccess(); + } + }; + Set assignment = assignPartitionsAndSeekToNext(consumer, topic); // Main loop. @@ -426,8 +459,10 @@ public void run() final SegmentIdentifier identifier = driver.add( row, sequenceNames.get(record.partition()), - committerSupplier - ); + committerSupplier, + publisher, + false + ).lhs; if (identifier == null) { // Failure to allocate segment puts determinism at risk, bail out to be safe. @@ -480,40 +515,7 @@ public void run() status = Status.PUBLISHING; } - final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() - { - @Override - public boolean publishSegments(Set segments, Object commitMetadata) throws IOException - { - final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( - ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), - KafkaPartitions.class - ); - - // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { - throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); - } - - final SegmentTransactionalInsertAction action; - - if (ioConfig.isUseTransaction()) { - action = new SegmentTransactionalInsertAction( - segments, - new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), - new KafkaDataSourceMetadata(finalPartitions) - ); - } else { - action = new SegmentTransactionalInsertAction(segments, null, null); - } - - log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); - - return toolbox.getTaskActionClient().submit(action).isSuccess(); - } - }; - - final SegmentsAndMetadata published = driver.finish(publisher, committerSupplier.get()); + final SegmentsAndMetadata published = driver.publishAndWaitHandoff(publisher, committerSupplier.get()); if (published == null) { throw new ISE("Transaction failure publishing segments, aborting"); } else { @@ -853,6 +855,7 @@ private FiniteAppenderatorDriver newDriver( new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), tuningConfig.getMaxRowsPerSegment(), + Long.MAX_VALUE, // KafkaIndexTask doesn't support this parameter yet tuningConfig.getHandoffConditionTimeout(), metrics ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index f02eb5a923b0..080ab3cf876c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -104,6 +104,13 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @Override + public long getMaxPersistedSegmentsBytes() + { + // This option is not supported yet + throw new UnsupportedOperationException(); + } + @JsonProperty public int getMaxRowsPerSegment() { diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java index de82223c5161..b25d88674185 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java @@ -19,6 +19,7 @@ package io.druid.indexing.appenderator; +import io.druid.data.input.InputRow; import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.segment.indexing.DataSchema; @@ -45,6 +46,7 @@ public ActionBasedSegmentAllocator( @Override public SegmentIdentifier allocate( final DateTime timestamp, + final InputRow row, final String sequenceName, final String previousSegmentId ) throws IOException diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 75a86d8b84bb..8b548640821c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -32,10 +32,8 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import io.druid.common.utils.JodaUtils; @@ -43,9 +41,7 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.Rows; import io.druid.guice.annotations.Smile; -import io.druid.hll.HyperLogLogCollector; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskLock; @@ -57,8 +53,7 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.granularity.Granularity; -import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.DruidMetrics; @@ -85,10 +80,9 @@ import io.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.HashBasedNumberedShardSpec; +import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; -import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; -import io.druid.timeline.partition.ShardSpecLookup; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -96,10 +90,14 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.SortedSet; +import java.util.stream.Collectors; public class IndexTask extends AbstractTask { @@ -189,12 +187,12 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); } - final Map> shardSpecs = determineShardSpecs(toolbox, firehoseFactory); + final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory); final String version; final DataSchema dataSchema; if (determineIntervals) { - Interval interval = JodaUtils.umbrellaInterval(shardSpecs.keySet()); + Interval interval = JodaUtils.umbrellaInterval(shardSpecs.getIntervals()); TaskLock lock = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval)); version = lock.getVersion(); dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( @@ -202,7 +200,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .getGranularitySpec() .withIntervals( JodaUtils.condenseIntervals( - shardSpecs.keySet() + shardSpecs.getIntervals() ) ) ); @@ -222,30 +220,49 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception * Determines the number of shards for each interval using a hash of queryGranularity timestamp + all dimensions (i.e * hash-based partitioning). In the future we may want to also support single-dimension partitioning. */ - private Map> determineShardSpecs( + private ShardSpecs determineShardSpecs( final TaskToolbox toolbox, final FirehoseFactory firehoseFactory ) throws IOException { final ObjectMapper jsonMapper = toolbox.getObjectMapper(); final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - final Granularity queryGranularity = granularitySpec.getQueryGranularity(); - final boolean determineNumPartitions = ingestionSchema.getTuningConfig().getNumShards() == null; - final boolean determineIntervals = !ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .isPresent(); + final boolean fixedNumPartitions = ingestionSchema.getTuningConfig().getNumShards() != null; + final boolean fixedIntervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals() + .isPresent() + && !ingestionSchema.getTuningConfig().isForceExtendableShardSpecs() + && !ingestionSchema.getIOConfig().isAppendToExisting(); + + final Set intervals; + if (fixedIntervals) { + log.info("intervals provided, skipping determine partition scan"); + intervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals() + .get(); + } else { + // determine intervals containing data + log.info("Determining intervals"); + intervals = new HashSet<>(); + long determineIntervalsStartMillis = System.currentTimeMillis(); - final Map> shardSpecs = Maps.newHashMap(); + try (final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser())) { + while (firehose.hasMore()) { + final InputRow inputRow = firehose.nextRow(); + final Interval interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); + + intervals.add(interval); + } + } + + log.info("Found intervals in %,dms", System.currentTimeMillis() - determineIntervalsStartMillis); + } - // if we were given number of shards per interval and the intervals, we don't need to scan the data - if (!determineNumPartitions && !determineIntervals) { - log.info("numShards and intervals provided, skipping determine partition scan"); - final SortedSet intervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .get(); + if (fixedNumPartitions) { final int numShards = ingestionSchema.getTuningConfig().getNumShards(); + final Map> intervalToShardSpecs = new HashMap<>(); for (Interval interval : intervals) { final List intervalShardSpecs = Lists.newArrayListWithCapacity(numShards); @@ -256,97 +273,78 @@ private Map> determineShardSpecs( } else { intervalShardSpecs.add(NoneShardSpec.instance()); } - shardSpecs.put(interval, intervalShardSpecs); + intervalToShardSpecs.put(interval, intervalShardSpecs); } - return shardSpecs; - } - - // determine intervals containing data and prime HLL collectors - final Map> hllCollectors = Maps.newHashMap(); - int thrownAway = 0; + return new ShardSpecs() + { - log.info("Determining intervals and shardSpecs"); - long determineShardSpecsStartMillis = System.currentTimeMillis(); - try (final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser())) { - while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); + @Override + public Collection getIntervals() + { + return intervalToShardSpecs.keySet(); + } - final Interval interval; - if (determineIntervals) { - interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); - } else { - final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - if (!optInterval.isPresent()) { - thrownAway++; - continue; - } - interval = optInterval.get(); + @Override + public boolean isExtendable() + { + return false; } - if (!determineNumPartitions) { - // we don't need to determine partitions but we still need to determine intervals, so add an Optional.absent() - // for the interval and don't instantiate a HLL collector - if (!hllCollectors.containsKey(interval)) { - hllCollectors.put(interval, Optional.absent()); + @Override + public ShardSpec getShardSpec(Interval interval, long timestamp, InputRow row) + { + final List shardSpecs = intervalToShardSpecs.get(interval); + if (shardSpecs == null || shardSpecs.isEmpty()) { + throw new ISE("Failed to get shardSpec for interval[%s]", interval); } - continue; + return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(timestamp, row); } - if (!hllCollectors.containsKey(interval)) { - hllCollectors.put(interval, Optional.of(HyperLogLogCollector.makeLatestCollector())); + @Override + public void updateShardSpec(Interval interval) + { + // do nothing + } + }; + } else { + final Map shardSpecMap = intervals.stream() + .collect(Collectors.toMap(interval -> interval, key -> new LinearShardSpec(0))); + return new ShardSpecs() + { + @Override + public Collection getIntervals() + { + return shardSpecMap.keySet(); } - List groupKey = Rows.toGroupKey( - queryGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), - inputRow - ); - hllCollectors.get(interval).get().add(hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asBytes()); - } - } - - if (thrownAway > 0) { - log.warn("Unable to to find a matching interval for [%,d] events", thrownAway); - } - - final ImmutableSortedMap> sortedMap = ImmutableSortedMap.copyOf( - hllCollectors, - Comparators.intervalsByStartThenEnd() - ); - - for (final Map.Entry> entry : sortedMap.entrySet()) { - final Interval interval = entry.getKey(); - final Optional collector = entry.getValue(); + @Override + public boolean isExtendable() + { + return true; + } - final int numShards; - if (determineNumPartitions) { - final long numRows = new Double(collector.get().estimateCardinality()).longValue(); - numShards = (int) Math.ceil((double) numRows / ingestionSchema.getTuningConfig().getTargetPartitionSize()); - log.info("Estimated [%,d] rows of data for interval [%s], creating [%,d] shards", numRows, interval, numShards); - } else { - numShards = ingestionSchema.getTuningConfig().getNumShards(); - log.info("Creating [%,d] shards for interval [%s]", numShards, interval); - } + @Override + public ShardSpec getShardSpec(Interval interval, long timestamp, InputRow row) + { + return shardSpecMap.get(interval); + } - final List intervalShardSpecs = Lists.newArrayListWithCapacity(numShards); - if (numShards > 1) { - for (int i = 0; i < numShards; i++) { - intervalShardSpecs.add(new HashBasedNumberedShardSpec(i, numShards, null, jsonMapper)); + @Override + public void updateShardSpec(Interval interval) + { + final LinearShardSpec previous = shardSpecMap.get(interval); + Preconditions.checkNotNull(previous, "previous shardSpec for interval[%s]", interval); + shardSpecMap.put(interval, new LinearShardSpec(previous.getPartitionNum() + 1)); } - } else { - intervalShardSpecs.add(NoneShardSpec.instance()); - } - shardSpecs.put(interval, intervalShardSpecs); + }; } - log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); - - return shardSpecs; } private boolean generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, - final Map> shardSpecs, + final ShardSpecs shardSpecs, final String version, final FirehoseFactory firehoseFactory ) throws IOException, InterruptedException @@ -357,7 +355,6 @@ private boolean generateAndPublishSegments( dataSchema, new RealtimeIOConfig(null, null, null), null ); final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); - final Map sequenceNameToShardSpecMap = Maps.newHashMap(); if (toolbox.getMonitorScheduler() != null) { toolbox.getMonitorScheduler().addMonitor( @@ -372,24 +369,18 @@ dataSchema, new RealtimeIOConfig(null, null, null), null if (ingestionSchema.getIOConfig().isAppendToExisting()) { segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); } else { - segmentAllocator = new SegmentAllocator() - { - @Override - public SegmentIdentifier allocate(DateTime timestamp, String sequenceName, String previousSegmentId) - throws IOException - { - Optional interval = granularitySpec.bucketInterval(timestamp); - if (!interval.isPresent()) { - throw new ISE("Could not find interval for timestamp [%s]", timestamp); - } - - ShardSpec shardSpec = sequenceNameToShardSpecMap.get(sequenceName); - if (shardSpec == null) { - throw new ISE("Could not find ShardSpec for sequenceName [%s]", sequenceName); - } + segmentAllocator = (timestamp, row, sequenceName, previousSegmentId) -> { + Optional interval = granularitySpec.bucketInterval(timestamp); + if (!interval.isPresent()) { + throw new ISE("Could not find interval for timestamp [%s]", timestamp); + } - return new SegmentIdentifier(getDataSource(), interval.get(), version, shardSpec); + ShardSpec shardSpec = shardSpecs.getShardSpec(interval.get(), timestamp.getMillis(), row); + if (shardSpec == null) { + throw new ISE("Could not find ShardSpec for sequenceName [%s]", sequenceName); } + + return new SegmentIdentifier(getDataSource(), interval.get(), version, shardSpec); }; } @@ -399,12 +390,16 @@ public SegmentIdentifier allocate(DateTime timestamp, String sequenceName, Strin appenderator, toolbox, segmentAllocator, - fireDepartmentMetrics + fireDepartmentMetrics, + ingestionSchema.getTuningConfig() ); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser()) ) { final Supplier committerSupplier = Committers.supplierFromFirehose(firehose); - final Map shardSpecLookups = Maps.newHashMap(); + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments, null, null); + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; if (driver.startJob() != null) { driver.clear(); @@ -422,36 +417,15 @@ public SegmentIdentifier allocate(DateTime timestamp, String sequenceName, Strin } final Interval interval = optInterval.get(); - if (!shardSpecLookups.containsKey(interval)) { - final List intervalShardSpecs = shardSpecs.get(interval); - if (intervalShardSpecs == null || intervalShardSpecs.isEmpty()) { - throw new ISE("Failed to get shardSpec for interval[%s]", interval); - } - shardSpecLookups.put(interval, intervalShardSpecs.get(0).getLookup(intervalShardSpecs)); - } - - final ShardSpec shardSpec = shardSpecLookups.get(interval) - .getShardSpec(inputRow.getTimestampFromEpoch(), inputRow); - - final String sequenceName = String.format("index_%s_%s_%d", interval, version, shardSpec.getPartitionNum()); - - if (!sequenceNameToShardSpecMap.containsKey(sequenceName)) { - final ShardSpec shardSpecForPublishing = ingestionSchema.getTuningConfig().isForceExtendableShardSpecs() - || ingestionSchema.getIOConfig().isAppendToExisting() - ? new NumberedShardSpec( - shardSpec.getPartitionNum(), - shardSpecs.get(interval).size() - ) - : shardSpec; - - sequenceNameToShardSpecMap.put(sequenceName, shardSpecForPublishing); - } - - final SegmentIdentifier identifier = driver.add(inputRow, sequenceName, committerSupplier); + final String sequenceName = Appenderators.getSequenceName(interval, version, shardSpecs.getShardSpec(interval, inputRow.getTimestampFromEpoch(), inputRow)); + final Pair> pair = driver.add(inputRow, sequenceName, committerSupplier, publisher, shardSpecs.isExtendable()); + final SegmentIdentifier identifier = pair.lhs; + final List publishedSegments = pair.rhs; if (identifier == null) { throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); } + publishedSegments.forEach(segmentId -> shardSpecs.updateShardSpec(segmentId.getInterval())); fireDepartmentMetrics.incrementProcessed(); } @@ -468,17 +442,7 @@ public SegmentIdentifier allocate(DateTime timestamp, String sequenceName, Strin driver.persist(committerSupplier.get()); } - final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() - { - @Override - public boolean publishSegments(Set segments, Object commitMetadata) throws IOException - { - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments, null, null); - return toolbox.getTaskActionClient().submit(action).isSuccess(); - } - }; - - final SegmentsAndMetadata published = driver.finish(publisher, committerSupplier.get()); + final SegmentsAndMetadata published = driver.publishAndWaitHandoff(publisher, committerSupplier.get()); if (published == null) { log.error("Failed to publish segments, aborting!"); return false; @@ -520,7 +484,8 @@ private FiniteAppenderatorDriver newDriver( final Appenderator appenderator, final TaskToolbox toolbox, final SegmentAllocator segmentAllocator, - final FireDepartmentMetrics metrics + final FireDepartmentMetrics metrics, + final IndexTuningConfig tuningConfig ) { return new FiniteAppenderatorDriver( @@ -529,12 +494,55 @@ private FiniteAppenderatorDriver newDriver( new NoopSegmentHandoffNotifierFactory(), // don't wait for handoff since we don't serve queries new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), - Integer.MAX_VALUE, // rows for a partition is already determined by the shardSpec + // If targetPartitionSize is null, numShards must be set which means intervals are already partitioned with + // proper shardSpecs. See determineShardSpecs(). + tuningConfig.getTargetPartitionSize() == null ? Integer.MAX_VALUE : tuningConfig.getTargetPartitionSize(), + tuningConfig.getMaxPersistedSegmentsBytes(), 0, metrics ); } + /** + * This interface represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. The most + * important method is {@link #updateShardSpec(Interval)} which updates the map according to the type of shardSpec. + */ + private interface ShardSpecs + { + /** + * Return the key set of the underlying map. + * + * @return a set of intervals + */ + Collection getIntervals(); + + /** + * Indicate that the type of shardSpecs is extendable like {@link LinearShardSpec}. + * + * @return true if the type of shardSpecs is extendable + */ + boolean isExtendable(); + + /** + * Return a shardSpec for the given interval, timestamp and input row. + * + * @param interval interval for shardSpec + * @param timestamp timestamp of input row + * @param row input row + * @return a shardSpec + */ + ShardSpec getShardSpec(Interval interval, long timestamp, InputRow row); + + /** + * Update the shardSpec of the given interval. When the type of shardSpecs is extendable, this method must update + * the shardSpec properly. For example, if the {@link LinearShardSpec} is used, an implementation of this method + * may replace the shardSpec of the given interval with a new one having a greater partitionNum. + * + * @param interval interval for shardSpec to be updated + */ + void updateShardSpec(Interval interval); + } + public static class IndexIngestionSpec extends IngestionSpec { private final DataSchema dataSchema; @@ -554,7 +562,7 @@ public IndexIngestionSpec( this.ioConfig = ioConfig; this.tuningConfig = tuningConfig == null ? - new IndexTuningConfig(null, null, null, null, null, null, null, null, (File) null) + new IndexTuningConfig(null, null, null, null, null, null, null, null, null, (File) null) : tuningConfig; } @@ -625,6 +633,7 @@ public boolean isSkipFirehoseCaching() public static class IndexTuningConfig implements TuningConfig, AppenderatorConfig { private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75000; + private static final long DEFAULT_MAX_PERSISTED_SEGMENTS_BYTES = 1024 * 1024 * 1024; private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final boolean DEFAULT_BUILD_V9_DIRECTLY = true; @@ -635,6 +644,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final Integer targetPartitionSize; private final int maxRowsInMemory; + private final long maxPersistedSegmentsBytes; private final Integer numShards; private final IndexSpec indexSpec; private final File basePersistDirectory; @@ -647,6 +657,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi public IndexTuningConfig( @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxPersistedSegmentsBytes") @Nullable Long maxPersistedSegmentsBytes, @JsonProperty("rowFlushBoundary") @Nullable Integer rowFlushBoundary_forBackCompatibility, // DEPRECATED @JsonProperty("numShards") @Nullable Integer numShards, @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, @@ -659,6 +670,7 @@ public IndexTuningConfig( this( targetPartitionSize, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, + maxPersistedSegmentsBytes, numShards, indexSpec, maxPendingPersists, @@ -672,6 +684,7 @@ public IndexTuningConfig( private IndexTuningConfig( @Nullable Integer targetPartitionSize, @Nullable Integer maxRowsInMemory, + @Nullable Long maxPersistedSegmentsBytes, @Nullable Integer numShards, @Nullable IndexSpec indexSpec, @Nullable Integer maxPendingPersists, @@ -692,6 +705,9 @@ private IndexTuningConfig( ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize); this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + this.maxPersistedSegmentsBytes = maxPersistedSegmentsBytes == null + ? DEFAULT_MAX_PERSISTED_SEGMENTS_BYTES + : maxPersistedSegmentsBytes; this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; @@ -710,6 +726,7 @@ public IndexTuningConfig withBasePersistDirectory(File dir) return new IndexTuningConfig( targetPartitionSize, maxRowsInMemory, + maxPersistedSegmentsBytes, numShards, indexSpec, maxPendingPersists, @@ -733,6 +750,13 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @JsonProperty + @Override + public long getMaxPersistedSegmentsBytes() + { + return maxPersistedSegmentsBytes; + } + @JsonProperty public Integer getNumShards() { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 81e85c70dba1..0fe96baccdce 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -51,7 +51,7 @@ import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.HashBasedNumberedShardSpec; +import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; @@ -107,7 +107,7 @@ public void testDeterminePartitions() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, 2, null, false, false), + createIngestionSpec(tmpDir, null, 2, null, null, false, false), null, jsonMapper ); @@ -118,15 +118,13 @@ public void testDeterminePartitions() throws Exception Assert.assertEquals("test", segments.get(0).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(HashBasedNumberedShardSpec.class)); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getPartitions()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(HashBasedNumberedShardSpec.class)); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getPartitions()); } @Test @@ -145,7 +143,7 @@ public void testForceExtendableShardSpecs() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, 2, null, true, false), + createIngestionSpec(tmpDir, null, 2, null, null, true, false), null, jsonMapper ); @@ -156,15 +154,13 @@ public void testForceExtendableShardSpecs() throws Exception Assert.assertEquals("test", segments.get(0).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(NumberedShardSpec.class)); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getPartitions()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(NumberedShardSpec.class)); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getPartitions()); } @Test @@ -191,6 +187,7 @@ public void testWithArbitraryGranularity() throws Exception ), 10, null, + null, false, false ), @@ -227,6 +224,7 @@ public void testIntervalBucketing() throws Exception ), 50, null, + null, false, false ), @@ -254,7 +252,7 @@ public void testNumShardsProvided() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, null, 1, false, false), + createIngestionSpec(tmpDir, null, null, 1, null, false, false), null, jsonMapper ); @@ -285,7 +283,7 @@ public void testAppendToExisting() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, 2, null, false, true), + createIngestionSpec(tmpDir, null, 2, null, null, false, true), null, jsonMapper ); @@ -330,6 +328,7 @@ public void testIntervalNotSpecified() throws Exception ), 2, null, + null, false, false ), @@ -343,17 +342,69 @@ public void testIntervalNotSpecified() throws Exception Assert.assertEquals("test", segments.get(0).getDataSource()); Assert.assertEquals(new Interval("2014-01-01T00/PT1H"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(NoneShardSpec.class)); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); + + Assert.assertEquals("test", segments.get(1).getDataSource()); + Assert.assertEquals(new Interval("2014-01-01T01/PT1H"), segments.get(1).getInterval()); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertEquals(0, segments.get(1).getShardSpec().getPartitionNum()); + + Assert.assertEquals("test", segments.get(2).getDataSource()); + Assert.assertEquals(new Interval("2014-01-01T02/PT1H"), segments.get(2).getInterval()); + Assert.assertTrue(segments.get(2).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertEquals(0, segments.get(2).getShardSpec().getPartitionNum()); + } + + @Test + public void testWithSmallPersistedSegmentBytes() throws Exception + { + File tmpDir = temporaryFolder.newFolder(); + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + PrintWriter writer = new PrintWriter(tmpFile); + writer.println("2014-01-01T00:00:10Z,a,1"); + writer.println("2014-01-01T01:00:20Z,b,1"); + writer.println("2014-01-01T02:00:30Z,c,1"); + writer.close(); + + IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + tmpDir, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + null + ), + 2, + null, + 10L, + false, + false + ), + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + + Assert.assertEquals(3, segments.size()); + + Assert.assertEquals("test", segments.get(0).getDataSource()); + Assert.assertEquals(new Interval("2014-01-01T00/PT1H"), segments.get(0).getInterval()); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(new Interval("2014-01-01T01/PT1H"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(NoneShardSpec.class)); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(0, segments.get(1).getShardSpec().getPartitionNum()); Assert.assertEquals("test", segments.get(2).getDataSource()); Assert.assertEquals(new Interval("2014-01-01T02/PT1H"), segments.get(2).getInterval()); - Assert.assertTrue(segments.get(2).getShardSpec().getClass().equals(NoneShardSpec.class)); + Assert.assertTrue(segments.get(2).getShardSpec().getClass().equals(LinearShardSpec.class)); Assert.assertEquals(0, segments.get(2).getShardSpec().getPartitionNum()); } @@ -437,6 +488,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( GranularitySpec granularitySpec, Integer targetPartitionSize, Integer numShards, + Long persistedSegmentsBytes, boolean forceExtendableShardSpecs, boolean appendToExisting ) @@ -484,6 +536,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( new IndexTask.IndexTuningConfig( targetPartitionSize, 1, + persistedSegmentsBytes, null, numShards, indexSpec, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index c7ec7b849d31..7db77f87a734 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -187,7 +187,7 @@ public void testIndexTaskSerde() throws Exception jsonMapper ), new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true, true), - new IndexTask.IndexTuningConfig(10000, 10, 9999, null, indexSpec, 3, true, true, true) + new IndexTask.IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, true) ), null, jsonMapper @@ -252,7 +252,7 @@ public void testIndexTaskwithResourceSerde() throws Exception jsonMapper ), new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, 3, true, true, true) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true) ), null, jsonMapper diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index a4373d5be9a2..fd2770fd2643 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -653,7 +653,7 @@ public void testIndexTask() throws Exception mapper ), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, 3, true, true, true) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true) ), null, MAPPER @@ -711,7 +711,7 @@ public void testIndexTaskFailure() throws Exception mapper ), new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, 3, true, true, true) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true) ), null, MAPPER @@ -1070,7 +1070,7 @@ public void testResumeTasks() throws Exception mapper ), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, indexSpec, null, false, null, null) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null) ), null, MAPPER diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 640145d66cb5..a071d0345714 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -149,6 +149,13 @@ public int getMaxRowsInMemory() return maxRowsInMemory; } + @Override + public long getMaxPersistedSegmentsBytes() + { + // This option is not supported yet + throw new UnsupportedOperationException(); + } + @JsonProperty public Period getIntermediatePersistPeriod() { diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index be305c5c13ea..5225e6ec89e3 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -52,6 +52,8 @@ public interface Appenderator extends QuerySegmentWalker, Closeable */ Object startJob(); + long getPersistedBytes(); + /** * Add a row. Must not be called concurrently from multiple threads. *

diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java index 4b2c99f644c0..dd6c6bd04b7b 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -30,6 +30,8 @@ public interface AppenderatorConfig int getMaxRowsInMemory(); + long getMaxPersistedSegmentsBytes(); + int getMaxPendingPersists(); Period getIntermediatePersistPeriod(); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 1c3e2794b94d..07f8a47f1f30 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -121,6 +121,8 @@ public class AppenderatorImpl implements Appenderator private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; + private long persistedBytes; + public AppenderatorImpl( DataSchema schema, AppenderatorConfig tuningConfig, @@ -166,6 +168,12 @@ public String getDataSource() return schema.getDataSource(); } + @Override + public long getPersistedBytes() + { + return persistedBytes; + } + @Override public Object startJob() { @@ -923,7 +931,17 @@ public Object apply(@Nullable Object input) } if (removeOnDiskData) { - removeDirectory(computePersistDir(identifier)); + final File persistDir = computePersistDir(identifier); + long persistFilesSize = 0; + for (FireHydrant hydrant : sink) { + final File innerDir = new File(persistDir, String.valueOf(hydrant.getCount())); + if (innerDir.exists()) { + persistFilesSize += FileUtils.sizeOfDirectory(innerDir); + } + } + removeDirectory(persistDir); + persistedBytes -= persistFilesSize; + log.info("persistFilesSize[%d], persistedBytes[%d]", persistFilesSize, persistedBytes); } return null; @@ -1002,6 +1020,8 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdentifier identif new File(persistDir, String.valueOf(indexToPersist.getCount())), indexSpec ); + persistedBytes += FileUtils.sizeOfDirectory(persistedFile); + log.info("persistedFile[%d], persistedBytes[%d]", FileUtils.sizeOfDirectory(persistedFile), persistedBytes); indexToPersist.swapSegment( new QueryableIndexSegment( diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java index f978221e9558..41b1685118c0 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java @@ -31,6 +31,8 @@ import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; import java.io.IOException; import java.util.concurrent.ExecutorService; @@ -128,4 +130,8 @@ public boolean isAnnounced(DataSegment segment) ); } + public static String getSequenceName(Interval interval, String version, ShardSpec shardSpec) + { + return String.format("index_%s_%s_%d", interval, version, shardSpec.getPartitionNum()); + } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java index 8119f81f06ae..c0b3406939c7 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java @@ -35,10 +35,10 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; - import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.query.SegmentDescriptor; import io.druid.segment.realtime.FireDepartmentMetrics; @@ -49,12 +49,15 @@ import java.io.Closeable; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.Random; import java.util.Set; import java.util.TreeMap; +import java.util.stream.Collectors; /** * A FiniteAppenderatorDriver drives an Appenderator to index a finite stream of data. This class does not help you @@ -80,6 +83,8 @@ public class FiniteAppenderatorDriver implements Closeable private final long handoffConditionTimeout; private final FireDepartmentMetrics metrics; + private final long maxPersistedSegmentsBytes; + // All access to "activeSegments" and "lastSegmentId" must be synchronized on "activeSegments". // sequenceName -> start of segment interval -> segment we're currently adding data to @@ -111,6 +116,7 @@ public FiniteAppenderatorDriver( UsedSegmentChecker usedSegmentChecker, ObjectMapper objectMapper, int maxRowsPerSegment, + long maxPersistedSegmentsBytes, long handoffConditionTimeout, FireDepartmentMetrics metrics ) @@ -122,6 +128,7 @@ public FiniteAppenderatorDriver( this.usedSegmentChecker = Preconditions.checkNotNull(usedSegmentChecker, "usedSegmentChecker"); this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); this.maxRowsPerSegment = maxRowsPerSegment; + this.maxPersistedSegmentsBytes = maxPersistedSegmentsBytes; this.handoffConditionTimeout = handoffConditionTimeout; this.metrics = Preconditions.checkNotNull(metrics, "metrics"); } @@ -178,33 +185,69 @@ public void clear() throws InterruptedException } /** - * Add a row. Must not be called concurrently from multiple threads. + * Add a row. This method may internally incur persisting data added so far. Also, if too large data are persisted, + * it can incur publishing data. This method must not be called concurrently from multiple threads. * * @param row the row to add * @param sequenceName sequenceName for this row's segment * @param committerSupplier supplier of a committer associated with all data that has been added, including this row + * @param publisher a publisher to publish segments + * @param publish enable publishing or not * - * @return segment to which this row was added, or null if segment allocator returned null for this row + * @return a pair of a segmentIdentifier and a list of segmentIdentifiers. The lhs of the result pair represents + * segment to which this row was added, or null if segment allocator returned null for this row. The rhs of the result + * pair is the list identifiers of published segments * * @throws IOException if there is an I/O error while allocating or writing to a segment */ - public SegmentIdentifier add( + public Pair> add( final InputRow row, final String sequenceName, - final Supplier committerSupplier - ) throws IOException + final Supplier committerSupplier, + final TransactionalSegmentPublisher publisher, + final boolean publish + ) throws IOException, InterruptedException { Preconditions.checkNotNull(row, "row"); Preconditions.checkNotNull(sequenceName, "sequenceName"); Preconditions.checkNotNull(committerSupplier, "committerSupplier"); + Preconditions.checkNotNull(publisher, "publisher"); - final SegmentIdentifier identifier = getSegment(row.getTimestamp(), sequenceName); + final SegmentIdentifier identifier = getSegment(row.getTimestamp(), row, sequenceName); + final List movedOutSegments = new ArrayList<>(); if (identifier != null) { try { final int numRows = appenderator.add(identifier, row, wrapCommitterSupplier(committerSupplier)); - if (numRows >= maxRowsPerSegment) { - moveSegmentOut(sequenceName, ImmutableList.of(identifier)); + + if (publish && appenderator.getPersistedBytes() > maxPersistedSegmentsBytes) { + log.info("trigger publish. maxPersistedSegmentBytes[%d], appenderator.getPersistedBytes()[%d]", + maxPersistedSegmentsBytes, appenderator.getPersistedBytes()); + // publish segments generated so far + final SegmentsAndMetadata published = publishAndWaitHandoff(publisher, wrapCommitter(committerSupplier.get())); + + // find published activeSegments and move them out + final Map> sequenceToSegmentId = new HashMap<>(); + published.getSegments().forEach(segment -> { + final SegmentIdentifier moveOutSegmentId = SegmentIdentifier.fromDataSegment(segment); + final String moveOutSequence = Appenderators.getSequenceName( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec() + ); + + sequenceToSegmentId.computeIfAbsent(moveOutSequence, key -> new ArrayList<>()).add(moveOutSegmentId); + }); + sequenceToSegmentId.forEach((key, val) -> moveSegmentOut(key, val, true)); + movedOutSegments.addAll( + published.getSegments().stream() + .map(SegmentIdentifier::fromDataSegment) + .collect(Collectors.toList()) + ); + log.info("published. appenderator.getPersistedBytes()[%d]", appenderator.getPersistedBytes()); + } else if (numRows >= maxRowsPerSegment) { + moveSegmentOut(sequenceName, ImmutableList.of(identifier), false); + movedOutSegments.add(identifier); } } catch (SegmentNotWritableException e) { @@ -212,13 +255,13 @@ public SegmentIdentifier add( } } - return identifier; + return new Pair<>(identifier, movedOutSegments); } /** * Persist all data indexed through this driver so far. Blocks until complete. * - * Should be called after all data has been added through {@link #add(InputRow, String, Supplier)}. + * Should be called after all data has been added through {@link #add(InputRow, String, Supplier, TransactionalSegmentPublisher, boolean)} )}. * * @param committer committer representing all data that has been added so far * @@ -245,8 +288,8 @@ public Object persist(final Committer committer) throws InterruptedException * Publish all data indexed through this driver so far, and waits for it to be handed off. Blocks until complete. * Retries forever on transient failures, but may exit early on permanent failures. * - * Should be called after all data has been added and persisted through {@link #add(InputRow, String, Supplier)} and - * {@link #persist(Committer)}. + * Should be called after all data has been added and persisted through {@link #add(InputRow, String, Supplier, TransactionalSegmentPublisher, boolean)} + * and {@link #persist(Committer)}. * * @param publisher publisher to use for this set of segments * @param committer committer representing all data that has been added so far @@ -254,7 +297,7 @@ public Object persist(final Committer committer) throws InterruptedException * @return segments and metadata published if successful, or null if segments could not be handed off due to * transaction failure with commit metadata. */ - public SegmentsAndMetadata finish( + public SegmentsAndMetadata publishAndWaitHandoff( final TransactionalSegmentPublisher publisher, final Committer committer ) throws InterruptedException @@ -335,7 +378,11 @@ private SegmentIdentifier getActiveSegment(final DateTime timestamp, final Strin * * @throws IOException if an exception occurs while allocating a segment */ - private SegmentIdentifier getSegment(final DateTime timestamp, final String sequenceName) throws IOException + private SegmentIdentifier getSegment( + final DateTime timestamp, + final InputRow row, + final String sequenceName + ) throws IOException { synchronized (activeSegments) { final SegmentIdentifier existing = getActiveSegment(timestamp, sequenceName); @@ -346,6 +393,7 @@ private SegmentIdentifier getSegment(final DateTime timestamp, final String sequ final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); final SegmentIdentifier newSegment = segmentAllocator.allocate( timestamp, + row, sequenceName, lastSegmentIds.get(sequenceName) ); @@ -353,6 +401,11 @@ private SegmentIdentifier getSegment(final DateTime timestamp, final String sequ if (newSegment != null) { final Long key = newSegment.getInterval().getStartMillis(); + // If an interval is partitioned with the HashBasedNumberedShardSpec, that partition spec is fixed and cannot + // be modified. It means, segmentAllocator cannot create a new segmentIdentifier and may return an existing + // one. To avoid this, activeSegments are never moved out by setting maxRowsPerSegment to Integer.MAX_VALUE + // and disallowing segment publish in the middle of data ingestion. See IndexTask.newDriver() and + // IndexTask.determineShardSpecs(). for (SegmentIdentifier identifier : appenderator.getSegments()) { if (identifier.equals(newSegment)) { throw new ISE( @@ -367,7 +420,10 @@ private SegmentIdentifier getSegment(final DateTime timestamp, final String sequ if (activeSegmentsForSequence == null) { activeSegments.put(sequenceName, Maps.newTreeMap()); } - activeSegments.get(sequenceName).put(key, newSegment); + final SegmentIdentifier previous = activeSegments.get(sequenceName).put(key, newSegment); + if (previous != null) { + log.error("previous[%s] is not null for key[%d] and new val[%s]", previous, key, newSegment); + } lastSegmentIds.put(sequenceName, newSegment.getIdentifierAsString()); } else { // Well, we tried. @@ -382,21 +438,30 @@ private SegmentIdentifier getSegment(final DateTime timestamp, final String sequ /** * Move a set of identifiers out from "active", making way for newer segments. */ - private void moveSegmentOut(final String sequenceName, final List identifiers) + private boolean moveSegmentOut( + final String sequenceName, + final List identifiers, + boolean ignoreAbsent + ) { synchronized (activeSegments) { final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); if (activeSegmentsForSequence == null) { - throw new ISE("WTF?! Asked to remove segments for sequenceName[%s] which doesn't exist...", sequenceName); + if (ignoreAbsent) { + return false; + } else { + throw new ISE("WTF?! Asked to remove segments for sequenceName[%s] which doesn't exist...", sequenceName); + } } for (final SegmentIdentifier identifier : identifiers) { log.info("Moving segment[%s] out of active list.", identifier); final long key = identifier.getInterval().getStartMillis(); - if (activeSegmentsForSequence.remove(key) != identifier) { + if (!activeSegmentsForSequence.remove(key).equals(identifier)) { throw new ISE("WTF?! Asked to remove segment[%s] that didn't exist...", identifier); } } + return true; } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java index 94b5ba245ff6..759be21a3074 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java @@ -19,6 +19,7 @@ package io.druid.segment.realtime.appenderator; +import io.druid.data.input.InputRow; import org.joda.time.DateTime; import java.io.IOException; @@ -36,6 +37,7 @@ public interface SegmentAllocator */ SegmentIdentifier allocate( DateTime timestamp, + InputRow row, String sequenceName, String previousSegmentId ) throws IOException; diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index 16484c48cdae..316e86b6a199 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -129,7 +129,7 @@ public void testSimpleIngestion() throws Exception { SegmentIdentifier si = plumber.getSegmentsView().values().toArray(new SegmentIdentifier[1])[0]; Assert.assertEquals(3, - appenderator.getRowCount(si)); + appenderator.getRowCount(si)); appenderator.clear(); Assert.assertTrue(appenderator.getSegments().isEmpty()); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 8fb6619dd831..2a4490056a04 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -240,6 +240,48 @@ public void run() } } + @Test(timeout = 10000L) + public void testPersistedBytes() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(3)) { + final Appenderator appenderator = tester.getAppenderator(); + final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); + final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); + + Assert.assertEquals(0, appenderator.getPersistedBytes()); + appenderator.startJob(); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + + appenderator.persistAll(committerSupplier.get()).get(); + Assert.assertEquals(4456, appenderator.getPersistedBytes()); + appenderator.drop(IDENTIFIERS.get(0)).get(); + Assert.assertEquals(2228, appenderator.getPersistedBytes()); + appenderator.drop(IDENTIFIERS.get(1)).get(); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + + appenderator.add(IDENTIFIERS.get(2), IR("2001", "bar", 1), committerSupplier); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + appenderator.add(IDENTIFIERS.get(2), IR("2001", "baz", 1), committerSupplier); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + appenderator.add(IDENTIFIERS.get(2), IR("2001", "qux", 1), committerSupplier); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + appenderator.add(IDENTIFIERS.get(2), IR("2001", "bob", 1), committerSupplier); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + + appenderator.persistAll(committerSupplier.get()).get(); + Assert.assertEquals(4522, appenderator.getPersistedBytes()); + appenderator.drop(IDENTIFIERS.get(2)).get(); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + + appenderator.close(); + Assert.assertEquals(0, appenderator.getPersistedBytes()); + } + } + @Test public void testQueryByIntervals() throws Exception { diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java index acf98ef38846..26ad0fa052b6 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java @@ -33,6 +33,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.query.SegmentDescriptor; @@ -65,8 +66,9 @@ public class FiniteAppenderatorDriverTest private static final String DATA_SOURCE = "foo"; private static final String VERSION = "abc123"; private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - private static final int MAX_ROWS_IN_MEMORY = 100; + private static final int MAX_ROWS_IN_MEMORY = 4; private static final int MAX_ROWS_PER_SEGMENT = 3; + private static final long MAX_PERSISTED_SEGMENTS_BYTES = 1024; private static final long HANDOFF_CONDITION_TIMEOUT = 0; private static final List ROWS = Arrays.asList( @@ -103,6 +105,7 @@ public void setUp() new TestUsedSegmentChecker(), OBJECT_MAPPER, MAX_ROWS_PER_SEGMENT, + MAX_PERSISTED_SEGMENTS_BYTES, HANDOFF_CONDITION_TIMEOUT, new FireDepartmentMetrics() ); @@ -124,10 +127,10 @@ public void testSimple() throws Exception for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); - Assert.assertNotNull(driver.add(ROWS.get(i), "dummy", committerSupplier)); + Assert.assertNotNull(driver.add(ROWS.get(i), "dummy", committerSupplier, makeOkPublisher(), false)); } - final SegmentsAndMetadata segmentsAndMetadata = driver.finish( + final SegmentsAndMetadata segmentsAndMetadata = driver.publishAndWaitHandoff( makeOkPublisher(), committerSupplier.get() ); @@ -162,14 +165,70 @@ public void testMaxRowsPerSegment() throws Exception 2.0 ) ); - Assert.assertNotNull(driver.add(row, "dummy", committerSupplier)); + Assert.assertNotNull(driver.add(row, "dummy", committerSupplier, makeOkPublisher(), false)); } - final SegmentsAndMetadata segmentsAndMetadata = driver.finish(makeOkPublisher(), committerSupplier.get()); + final SegmentsAndMetadata segmentsAndMetadata = driver.publishAndWaitHandoff( + makeOkPublisher(), + committerSupplier.get() + ); Assert.assertEquals(numSegments, segmentsAndMetadata.getSegments().size()); Assert.assertEquals(numSegments * MAX_ROWS_PER_SEGMENT, segmentsAndMetadata.getCommitMetadata()); } + @Test + public void testMaxPersistedSegmentsBytes() throws Exception + { + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < 7; i++) { // i = 7 + committerSupplier.setMetadata(i + 1); + InputRow row = new MapBasedInputRow( + new DateTime("2000T01"), + ImmutableList.of("dim2"), + ImmutableMap.of( + "dim2", + String.format("bar-%d", i), + "met1", + 2.0 + ) + ); + Assert.assertNotNull(driver.add(row, "dummy", committerSupplier, makeOkPublisher(), true)); + } + + committerSupplier.setMetadata(8); + InputRow row = new MapBasedInputRow( + new DateTime("2000T01"), + ImmutableList.of("dim2"), + ImmutableMap.of( + "dim2", + String.format("bar-%d", 7), + "met1", + 2.0 + ) + ); + final Pair> pair = driver.add( + row, + "dummy", + committerSupplier, + makeOkPublisher(), + true + ); + final SegmentIdentifier added = pair.lhs; + final List movedOut = pair.rhs; + + Assert.assertNotNull(added); + Assert.assertEquals(3, movedOut.size()); + + final SegmentsAndMetadata segmentsAndMetadata = driver.publishAndWaitHandoff( + makeOkPublisher(), + committerSupplier.get() + ); + Assert.assertEquals(0, segmentsAndMetadata.getSegments().size()); + Assert.assertEquals(8, segmentsAndMetadata.getCommitMetadata()); + } + private Set asIdentifiers(Iterable segments) { return ImmutableSet.copyOf( @@ -244,6 +303,7 @@ public TestSegmentAllocator(String dataSource, Granularity granularity) @Override public SegmentIdentifier allocate( final DateTime timestamp, + final InputRow row, final String sequenceName, final String previousSegmentId ) throws IOException From bfdbac9242c852f5126c2d0f885f4e037ad1553a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 2 May 2017 23:27:28 +0900 Subject: [PATCH 02/15] Remove unnecessary logs --- .../segment/realtime/appenderator/AppenderatorImpl.java | 2 -- .../realtime/appenderator/FiniteAppenderatorDriver.java | 8 +------- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 07f8a47f1f30..b1a189fc4a89 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -941,7 +941,6 @@ public Object apply(@Nullable Object input) } removeDirectory(persistDir); persistedBytes -= persistFilesSize; - log.info("persistFilesSize[%d], persistedBytes[%d]", persistFilesSize, persistedBytes); } return null; @@ -1021,7 +1020,6 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdentifier identif indexSpec ); persistedBytes += FileUtils.sizeOfDirectory(persistedFile); - log.info("persistedFile[%d], persistedBytes[%d]", FileUtils.sizeOfDirectory(persistedFile), persistedBytes); indexToPersist.swapSegment( new QueryableIndexSegment( diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java index c0b3406939c7..9b546d5f4f8e 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java @@ -221,8 +221,6 @@ public Pair> add( final int numRows = appenderator.add(identifier, row, wrapCommitterSupplier(committerSupplier)); if (publish && appenderator.getPersistedBytes() > maxPersistedSegmentsBytes) { - log.info("trigger publish. maxPersistedSegmentBytes[%d], appenderator.getPersistedBytes()[%d]", - maxPersistedSegmentsBytes, appenderator.getPersistedBytes()); // publish segments generated so far final SegmentsAndMetadata published = publishAndWaitHandoff(publisher, wrapCommitter(committerSupplier.get())); @@ -244,7 +242,6 @@ public Pair> add( .map(SegmentIdentifier::fromDataSegment) .collect(Collectors.toList()) ); - log.info("published. appenderator.getPersistedBytes()[%d]", appenderator.getPersistedBytes()); } else if (numRows >= maxRowsPerSegment) { moveSegmentOut(sequenceName, ImmutableList.of(identifier), false); movedOutSegments.add(identifier); @@ -420,10 +417,7 @@ private SegmentIdentifier getSegment( if (activeSegmentsForSequence == null) { activeSegments.put(sequenceName, Maps.newTreeMap()); } - final SegmentIdentifier previous = activeSegments.get(sequenceName).put(key, newSegment); - if (previous != null) { - log.error("previous[%s] is not null for key[%d] and new val[%s]", previous, key, newSegment); - } + activeSegments.get(sequenceName).put(key, newSegment); lastSegmentIds.put(sequenceName, newSegment.getIdentifierAsString()); } else { // Well, we tried. From 3ee09d2f2b4b29d4980b4832e22cc2b2b2dff504 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 7 May 2017 12:32:39 +0900 Subject: [PATCH 03/15] Address comments --- .../druid/indexing/common/task/IndexTask.java | 21 +++++++++++-------- .../realtime/appenderator/Appenderator.java | 6 ++++++ .../FiniteAppenderatorDriver.java | 2 ++ 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 8b548640821c..e12ee99b458c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -292,13 +292,13 @@ public boolean isExtendable() } @Override - public ShardSpec getShardSpec(Interval interval, long timestamp, InputRow row) + public ShardSpec getShardSpec(Interval interval, InputRow row) { final List shardSpecs = intervalToShardSpecs.get(interval); if (shardSpecs == null || shardSpecs.isEmpty()) { throw new ISE("Failed to get shardSpec for interval[%s]", interval); } - return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(timestamp, row); + return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(row.getTimestampFromEpoch(), row); } @Override @@ -325,7 +325,7 @@ public boolean isExtendable() } @Override - public ShardSpec getShardSpec(Interval interval, long timestamp, InputRow row) + public ShardSpec getShardSpec(Interval interval, InputRow row) { return shardSpecMap.get(interval); } @@ -375,7 +375,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null throw new ISE("Could not find interval for timestamp [%s]", timestamp); } - ShardSpec shardSpec = shardSpecs.getShardSpec(interval.get(), timestamp.getMillis(), row); + ShardSpec shardSpec = shardSpecs.getShardSpec(interval.get(), row); if (shardSpec == null) { throw new ISE("Could not find ShardSpec for sequenceName [%s]", sequenceName); } @@ -417,8 +417,12 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } final Interval interval = optInterval.get(); - final String sequenceName = Appenderators.getSequenceName(interval, version, shardSpecs.getShardSpec(interval, inputRow.getTimestampFromEpoch(), inputRow)); - final Pair> pair = driver.add(inputRow, sequenceName, committerSupplier, publisher, shardSpecs.isExtendable()); + final String sequenceName = Appenderators.getSequenceName( + interval, version, shardSpecs.getShardSpec(interval, inputRow) + ); + final Pair> pair = driver.add( + inputRow, sequenceName, committerSupplier, publisher, shardSpecs.isExtendable() + ); final SegmentIdentifier identifier = pair.lhs; final List publishedSegments = pair.rhs; @@ -524,14 +528,13 @@ private interface ShardSpecs boolean isExtendable(); /** - * Return a shardSpec for the given interval, timestamp and input row. + * Return a shardSpec for the given interval and input row. * * @param interval interval for shardSpec - * @param timestamp timestamp of input row * @param row input row * @return a shardSpec */ - ShardSpec getShardSpec(Interval interval, long timestamp, InputRow row); + ShardSpec getShardSpec(Interval interval, InputRow row); /** * Update the shardSpec of the given interval. When the type of shardSpecs is extendable, this method must update diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index 5225e6ec89e3..579f4f139040 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -52,6 +52,12 @@ public interface Appenderator extends QuerySegmentWalker, Closeable */ Object startJob(); + /** + * Returns the size of data currently stored in local storage. The persistedBytes is changed when + * {@link #persistAll(Committer)} and {@link #drop(SegmentIdentifier)} is called. + * + * @return the size of persisted data in bytes + */ long getPersistedBytes(); /** diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java index 9b546d5f4f8e..c7f2200f955b 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java @@ -236,6 +236,8 @@ public Pair> add( sequenceToSegmentId.computeIfAbsent(moveOutSequence, key -> new ArrayList<>()).add(moveOutSegmentId); }); + // Some of the published segments may be already moved out from activeSegments due to the maxRowsPerSegment limit. + // So, here the absent segments are simply ignored. sequenceToSegmentId.forEach((key, val) -> moveSegmentOut(key, val, true)); movedOutSegments.addAll( published.getSegments().stream() From f898776304e502b86bf8bcd4b639e7493c326d0a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 6 Jun 2017 16:11:10 +0900 Subject: [PATCH 04/15] Refactoring the patch according to #4292 and address comments --- .idea/inspectionProfiles/Druid.xml | 22 + .travis.yml | 29 +- INTELLIJ_SETUP.md | 61 + .../java/io/druid/data/input/Firehose.java | 9 +- .../io/druid/data/input/FirehoseFactory.java | 34 +- .../java/io/druid/data/input/MapBasedRow.java | 1 + .../AbstractTextFilesFirehoseFactory.java | 124 + .../druid/data/input/impl/CSVParseSpec.java | 49 +- .../data/input/impl/DelimitedParseSpec.java | 97 +- .../input/impl/FileIteratingFirehose.java | 62 +- .../PrefetchableTextFilesFirehoseFactory.java | 537 +++ .../data/input/impl/StringInputRowParser.java | 22 +- .../data/input/impl/TimeAndDimsParseSpec.java | 4 +- .../segment/loading/DataSegmentPusher.java | 30 + .../loading/DataSegmentPusherUtil.java | 66 - .../main/java/io/druid/utils/Runnables.java | 1 + .../data/input/impl/CSVParseSpecTest.java | 9 +- .../input/impl/DelimitedParseSpecTest.java | 29 +- .../input/impl/FileIteratingFirehoseTest.java | 165 +- .../input/impl/JavaScriptParseSpecTest.java | 11 +- .../druid/data/input/impl/ParseSpecTest.java | 15 +- ...fetchableTextFilesFirehoseFactoryTest.java | 420 +++ .../data/input/impl/RegexParseSpecTest.java | 8 +- .../java/io/druid/guice/PolyBindTest.java | 3 +- .../loading/DataSegmentPusherUtilTest.java | 56 - .../benchmark/FilterPartitionBenchmark.java | 6 +- .../FilteredAggregatorBenchmark.java | 18 +- .../benchmark/FlattenJSONBenchmarkUtil.java | 2 +- .../GroupByTypeInterfaceBenchmark.java | 6 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 8 +- .../BenchmarkColumnValueGenerator.java | 2 +- .../datagen/EnumeratedTreeDistribution.java | 2 +- .../datagen/RealRoundingDistribution.java | 2 +- .../datagen/SequentialDistribution.java | 4 +- .../IncrementalIndexReadBenchmark.java | 3 +- .../benchmark/query/GroupByBenchmark.java | 12 +- .../benchmark/query/QueryBenchmarkUtil.java | 5 +- .../benchmark/query/SearchBenchmark.java | 10 +- .../benchmark/query/SelectBenchmark.java | 3 +- .../query/SerializingQueryRunner.java | 6 +- .../druid/benchmark/query/SqlBenchmark.java | 4 +- .../benchmark/query/TimeseriesBenchmark.java | 10 +- .../druid/benchmark/query/TopNBenchmark.java | 8 +- .../benchmark/BenchmarkDataGeneratorTest.java | 2 +- .../java/io/druid/collections/IntegerSet.java | 11 +- .../bitmap/WrappedConciseBitmap.java | 2 +- .../collections/spatial/ImmutableNode.java | 1 + .../spatial/search/RectangularBound.java | 1 + ci/travis_script_integration.sh | 26 + codestyle/checkstyle.xml | 5 + common/pom.xml | 20 + .../annotations/SubclassesMustBePublic.java | 35 + ...lassesMustBePublicAnnotationProcessor.java | 51 + .../io/druid/collections/CountingMap.java | 61 - .../druid/common/guava/CombiningSequence.java | 3 + .../common/utils/ServletResourceUtils.java | 8 + .../timeline/VersionedIntervalTimeline.java | 54 +- .../javax.annotation.processing.Processor | 1 + .../io/druid/collections/CountingMapTest.java | 69 - .../collections/OrderedMergeSequenceTest.java | 5 +- .../common/guava/CombiningSequenceTest.java | 7 +- .../io/druid/common/utils/JodaUtilsTest.java | 2 +- .../java/io/druid/concurrent/ExecsTest.java | 1 + .../io/druid/guice/LifecycleScopeTest.java | 6 + .../VersionedIntervalTimelineTest.java | 21 +- distribution/pom.xml | 5 +- distribution/src/assembly/assembly.xml | 7 + docs/content/configuration/broker.md | 4 +- docs/content/configuration/historical.md | 2 +- docs/content/configuration/index.md | 3 +- docs/content/design/coordinator.md | 13 +- docs/content/design/indexing-service.md | 20 + .../development/extensions-contrib/azure.md | 10 +- .../extensions-contrib/cloudfiles.md | 9 +- .../development/extensions-contrib/google.md | 10 +- .../extensions-core/kafka-ingestion.md | 3 +- .../extensions-core/lookups-cached-global.md | 17 +- .../development/extensions-core/protobuf.md | 203 + .../content/development/extensions-core/s3.md | 13 +- docs/content/development/extensions.md | 1 + docs/content/ingestion/data-formats.md | 44 +- docs/content/ingestion/firehose.md | 22 + docs/content/ingestion/index.md | 7 - docs/content/ingestion/tasks.md | 4 +- docs/content/operations/metrics.md | 2 + docs/content/querying/dimensionspecs.md | 17 - docs/content/querying/groupbyquery.md | 1 + docs/content/querying/lookups.md | 22 + docs/content/querying/query-context.md | 1 + docs/content/querying/segmentmetadataquery.md | 7 +- docs/content/querying/select-query.md | 1 + .../druid/middleManager/runtime.properties | 2 +- .../druid/middleManager/runtime.properties | 2 +- .../quickstart/protobuf/kafka-metrics-pb.json | 71 + examples/quickstart/protobuf/metrics.desc | 15 + examples/quickstart/protobuf/metrics.proto | 11 + examples/quickstart/protobuf/metrics_pb2.py | 118 + examples/quickstart/protobuf/pb_publisher.py | 19 + .../TwitterSpritzerFirehoseFactory.java | 4 +- .../extendedset/intset/AbstractIntSet.java | 54 +- .../druid/extendedset/intset/ConciseSet.java | 48 +- .../extendedset/intset/ConciseSetUtils.java | 2 +- .../intset/ImmutableConciseSet.java | 4 +- .../io/druid/extendedset/intset/IntSet.java | 18 +- .../intset/ImmutableConciseSetTest.java | 5 +- .../io/druid/firehose/azure/AzureBlob.java | 9 + .../StaticAzureBlobStoreFirehoseFactory.java | 109 +- .../storage/azure/AzureDataSegmentPusher.java | 28 +- .../azure/AzureDataSegmentPusherTest.java | 5 +- .../cassandra/CassandraDataSegmentPusher.java | 11 +- .../firehose/cloudfiles/CloudFilesBlob.java | 10 + .../StaticCloudFilesFirehoseFactory.java | 109 +- .../CloudFilesDataSegmentPusher.java | 32 +- .../storage/cloudfiles/CloudFilesUtils.java | 7 - .../DistinctCountAggregatorFactory.java | 3 +- .../EmptyDistinctCountBufferAggregator.java | 1 + .../DistinctCountTimeseriesQueryTest.java | 5 +- .../DistinctCountTopNQueryTest.java | 3 +- .../rocketmq/RocketMQFirehoseFactory.java | 41 +- .../io/druid/firehose/google/GoogleBlob.java | 9 + .../StaticGoogleBlobStoreFirehoseFactory.java | 103 +- .../google/GoogleDataSegmentPusher.java | 42 +- .../google/GoogleDataSegmentPusherTest.java | 21 +- .../data/input/orc/OrcExtensionsModule.java | 4 +- .../input/orc/OrcHadoopInputRowParser.java | 1 + .../input/orc/OrcIndexGeneratorJobTest.java | 3 +- .../parquet/ParquetExtensionsModule.java | 6 +- .../parquet/avro/DruidParquetReadSupport.java | 1 + .../rabbitmq/RabbitMQFirehoseFactory.java | 3 +- .../java/io/druid/query/scan/ScanQuery.java | 18 +- .../query/scan/ScanQueryDruidModule.java | 2 + .../query/scan/ScanQueryLimitRowIterator.java | 7 +- .../query/scan/ScanQueryQueryToolChest.java | 18 +- .../query/scan/ScanQueryRunnerFactory.java | 12 +- .../query/scan/MultiSegmentScanQueryTest.java | 10 +- .../druid/query/scan/ScanQueryRunnerTest.java | 4 +- extensions-contrib/thrift-extensions/pom.xml | 20 + .../input/thrift/ThriftExtensionsModule.java | 4 +- .../TimestampAggregatorFactory.java | 3 +- .../aggregation/TimestampMinMaxModule.java | 4 +- .../io/druid/segment/MapVirtualColumn.java | 1 + .../druid/segment/MapVirtualColumnTest.java | 9 +- extensions-core/avro-extensions/pom.xml | 26 + .../data/input/avro/AvroValueInputFormat.java | 2 +- .../avro/SchemaRepoBasedAvroBytesDecoder.java | 10 +- .../input/AvroStreamInputRowParserTest.java | 1 + .../theta/EmptySketchBufferAggregator.java | 1 + .../theta/SketchEstimateWithErrorBounds.java | 2 +- extensions-core/hdfs-storage/pom.xml | 15 +- .../storage/hdfs/HdfsDataSegmentPusher.java | 57 +- .../loading/HdfsDataSegmentPullerTest.java | 18 +- .../HdfsFileTimestampVersionFinderTest.java | 57 +- .../hdfs/HdfsDataSegmentPusherTest.java | 299 +- .../histogram/ApproximateHistogram.java | 6 +- ...ApproximateHistogramAggregatorFactory.java | 3 +- .../ApproximateHistogramFoldingSerde.java | 1 + .../ApproximateHistogramGroupByQueryTest.java | 10 +- .../ApproximateHistogramTopNQueryTest.java | 197 +- .../sql/QuantileSqlAggregatorTest.java | 16 +- .../kafka/KafkaEightFirehoseFactory.java | 3 +- .../lookup/TestKafkaExtractionCluster.java | 2 + .../kafka-indexing-service/pom.xml | 8 +- .../druid/indexing/kafka/KafkaIOConfig.java | 15 +- .../druid/indexing/kafka/KafkaIndexTask.java | 168 +- .../indexing/kafka/KafkaIndexTaskClient.java | 4 +- .../indexing/kafka/KafkaTuningConfig.java | 15 +- .../kafka/supervisor/KafkaSupervisor.java | 135 +- .../supervisor/KafkaSupervisorIOConfig.java | 28 +- .../supervisor/KafkaSupervisorReport.java | 2 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 22 +- .../KafkaSupervisorTuningConfig.java | 4 +- .../indexing/kafka/KafkaIOConfigTest.java | 5 +- .../indexing/kafka/KafkaIndexTaskTest.java | 362 +- .../indexing/kafka/KafkaTuningConfigTest.java | 13 +- .../KafkaSupervisorIOConfigTest.java | 5 +- .../kafka/supervisor/KafkaSupervisorTest.java | 205 +- .../druid/indexing/kafka/test/TestBroker.java | 17 +- .../NamespaceLookupExtractorFactory.java | 6 + ...eCacheFactory.java => CacheGenerator.java} | 10 +- .../lookup/namespace/ExtractionNamespace.java | 4 +- ...pace.java => JdbcExtractionNamespace.java} | 8 +- ...space.java => UriExtractionNamespace.java} | 65 +- ...heFactory.java => JdbcCacheGenerator.java} | 23 +- .../namespace/NamespaceExtractionModule.java | 20 +- ...tory.java => StaticMapCacheGenerator.java} | 8 +- ...cheFactory.java => UriCacheGenerator.java} | 16 +- .../lookup/namespace/cache/CacheHandler.java | 1 + .../namespace/cache/CacheScheduler.java | 33 +- .../NamespaceLookupExtractorFactoryTest.java | 84 +- .../namespace/JSONFlatDataParserTest.java | 8 +- ...t.java => UriExtractionNamespaceTest.java} | 72 +- .../NamespacedExtractorModuleTest.java | 46 +- ....java => StaticMapCacheGeneratorTest.java} | 16 +- ...ryTest.java => UriCacheGeneratorTest.java} | 101 +- ...utorsTest.java => CacheSchedulerTest.java} | 48 +- ....java => JdbcExtractionNamespaceTest.java} | 38 +- .../NamespaceExtractionCacheManagersTest.java | 4 +- .../lookup/cache/loading/LoadingCache.java | 1 + extensions-core/protobuf-extensions/pom.xml | 123 + .../protobuf/ProtobufExtensionsModule.java | 50 +- .../protobuf/ProtobufInputRowParser.java | 151 + .../io.druid.initialization.DruidModule | 1 + .../input/protobuf/ProtoTestEventWrapper.java | 3258 +++++++++++++++++ .../protobuf/ProtobufInputRowParserTest.java | 170 + .../src/test/resources/ProtoTest.proto | 30 + .../src/test/resources/prototest.desc | Bin 0 -> 631 bytes .../firehose/s3/StaticS3FirehoseFactory.java | 194 +- .../s3/AWSSessionCredentialsAdapter.java | 1 + .../druid/storage/s3/S3DataSegmentMover.java | 3 +- .../druid/storage/s3/S3DataSegmentPuller.java | 1 + .../druid/storage/s3/S3DataSegmentPusher.java | 44 +- .../storage/s3/S3DataSegmentPusherConfig.java | 13 + .../java/io/druid/storage/s3/S3TaskLogs.java | 1 + .../java/io/druid/storage/s3/S3Utils.java | 6 +- .../s3/StaticS3FirehoseFactoryTest.java | 5 +- .../storage/s3/S3DataSegmentMoverTest.java | 2 +- .../s3/S3DataSegmentPusherConfigTest.java | 4 +- .../storage/s3/S3DataSegmentPusherTest.java | 2 +- .../variance/VarianceAggregatorFactory.java | 3 +- .../indexer/DetermineHashedPartitionsJob.java | 3 +- .../druid/indexer/DeterminePartitionsJob.java | 1 + .../indexer/HadoopDruidIndexerConfig.java | 13 +- .../io/druid/indexer/HadoopTuningConfig.java | 26 +- .../io/druid/indexer/IndexGeneratorJob.java | 20 +- .../java/io/druid/indexer/InputRowSerde.java | 43 +- .../main/java/io/druid/indexer/JobHelper.java | 145 +- .../indexer/hadoop/DatasourceInputFormat.java | 115 +- .../indexer/hadoop/FSSpideringIterator.java | 4 + .../partitions/AbstractPartitionsSpec.java | 3 + .../path/GranularUnprocessedPathSpec.java | 7 +- .../indexer/updater/HadoopConverterJob.java | 12 +- .../updater/HadoopDruidConverterConfig.java | 3 + .../indexer/BatchDeltaIngestionTest.java | 7 +- .../DetermineHashedPartitionsJobTest.java | 30 +- .../indexer/DeterminePartitionsJobTest.java | 7 +- .../indexer/HadoopDruidIndexerConfigTest.java | 237 +- .../druid/indexer/HadoopTuningConfigTest.java | 3 +- .../indexer/IndexGeneratorCombinerTest.java | 5 +- .../druid/indexer/IndexGeneratorJobTest.java | 19 +- .../io/druid/indexer/InputRowSerdeTest.java | 23 +- .../java/io/druid/indexer/JobHelperTest.java | 7 +- .../druid/indexer/UtilsCompressionTest.java | 23 +- .../test/java/io/druid/indexer/UtilsTest.java | 23 +- .../hadoop/DatasourceInputFormatTest.java | 104 + .../indexer/path/DatasourcePathSpecTest.java | 4 +- .../indexer/path/GranularityPathSpecTest.java | 3 +- .../updater/HadoopConverterJobTest.java | 7 +- .../ActionBasedSegmentAllocator.java | 4 +- .../indexing/common/SegmentLoaderFactory.java | 4 +- .../io/druid/indexing/common/TaskToolbox.java | 24 + .../indexing/common/TaskToolboxFactory.java | 5 + .../common/actions/LockAcquireAction.java | 1 + .../common/actions/LockListAction.java | 1 + .../common/actions/LockReleaseAction.java | 1 + .../common/actions/LockTryAcquireAction.java | 1 + .../common/actions/SegmentInsertAction.java | 1 + .../actions/SegmentListUnusedAction.java | 1 + .../common/actions/SegmentListUsedAction.java | 1 + .../actions/SegmentMetadataUpdateAction.java | 1 + .../common/actions/SegmentNukeAction.java | 1 + .../SegmentTransactionalInsertAction.java | 1 + .../indexing/common/config/TaskConfig.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 1 + .../indexing/common/task/HadoopTask.java | 21 +- .../druid/indexing/common/task/IndexTask.java | 613 +++- .../indexing/common/task/MergeTaskBase.java | 4 +- .../druid/indexing/common/task/NoopTask.java | 4 +- .../common/task/RealtimeIndexTask.java | 19 +- .../IngestSegmentFirehoseFactory.java | 2 +- .../indexing/overlord/ForkingTaskRunner.java | 1 + .../overlord/ImmutableWorkerInfo.java | 4 + .../indexing/overlord/RemoteTaskRunner.java | 12 +- .../druid/indexing/overlord/TaskMaster.java | 4 +- ...skBasedWorkerResourceManagementConfig.java | 5 + ...impleWorkerResourceManagementStrategy.java | 2 + .../overlord/http/OverlordRedirectInfo.java | 11 +- .../overlord/http/OverlordResource.java | 17 +- ...EqualDistributionWorkerSelectStrategy.java | 28 +- .../resources/indexer_static/console.html | 5 + .../indexer_static/js/console-0.0.1.js | 48 + .../indexing/common/TaskToolboxTest.java | 2 + .../actions/RemoteTaskActionClientTest.java | 43 +- .../indexing/common/task/IndexTaskTest.java | 281 +- .../common/task/RealtimeIndexTaskTest.java | 4 +- .../task/SameIntervalMergeTaskTest.java | 10 +- .../indexing/common/task/TaskSerdeTest.java | 10 +- .../IngestSegmentFirehoseFactoryTest.java | 10 +- ...estSegmentFirehoseFactoryTimelineTest.java | 3 +- ...kRunnerRunPendingTasksConcurrencyTest.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 37 +- .../autoscaling/EC2AutoScalerTest.java | 4 +- .../http/OverlordRedirectInfoTest.java | 25 +- .../overlord/http/OverlordResourceTest.java | 41 + .../indexing/overlord/http/OverlordTest.java | 6 +- ...lDistributionWorkerSelectStrategyTest.java | 118 +- .../supervisor/SupervisorManagerTest.java | 2 +- .../supervisor/SupervisorResourceTest.java | 2 +- .../test/TestDataSegmentAnnouncer.java | 6 - .../indexing/test/TestDataSegmentPusher.java | 8 + .../druid/indexing/test/TestServerView.java | 12 + .../worker/WorkerTaskMonitorTest.java | 2 +- integration-tests/docker/Dockerfile | 7 +- integration-tests/pom.xml | 40 +- .../druid/testing/guice/DruidTestModule.java | 3 +- .../druid/testing/utils/TestQueryHelper.java | 31 +- .../src/main/java/org/testng/TestNG.java | 12 +- .../io/druid/tests/indexer/ITIndexerTest.java | 3 +- .../indexer/ITKafkaIndexingServiceTest.java | 15 +- .../io/druid/tests/indexer/ITKafkaTest.java | 17 +- .../druid/tests/indexer/ITUnionQueryTest.java | 21 +- .../java/util/common/collect/CountingMap.java | 58 - .../common/concurrent/ScheduledExecutors.java | 8 +- .../common/granularity/GranularityType.java | 2 +- .../common/granularity/PeriodGranularity.java | 6 +- .../util/common/guava/DroppingIterable.java | 1 + .../util/common/guava/DroppingIterator.java | 3 + .../util/common/guava/FunctionalIterable.java | 1 + .../util/common/guava/FunctionalIterator.java | 3 + .../util/common/io/smoosh/FileSmoosher.java | 4 +- .../java/util/common/parsers/CSVParser.java | 93 +- .../util/common/parsers/DelimitedParser.java | 97 +- .../util/common/parsers/JSONPathParser.java | 3 +- .../util/common/parsers/JavaScriptParser.java | 1 + .../java/util/common/parsers/Parser.java | 19 +- .../java/util/common/parsers/RegexParser.java | 1 - .../common/parsers/ToLowerCaseParser.java | 6 + .../java/util/common/GranularityTest.java | 2 +- .../util/common/parsers/CSVParserTest.java | 48 +- .../common/parsers/DelimitedParserTest.java | 61 +- pom.xml | 108 +- processing/pom.xml | 20 +- .../data/input/ProtoBufInputRowParser.java | 129 - .../java/io/druid/guice/ExtensionsConfig.java | 10 + .../java/io/druid/query/AsyncQueryRunner.java | 6 +- .../main/java/io/druid/query/BaseQuery.java | 11 + .../io/druid/query/BySegmentQueryRunner.java | 14 +- .../query/BySegmentSkippingQueryRunner.java | 10 +- .../druid/query/CPUTimeMetricQueryRunner.java | 14 +- .../query/ChainedExecutionQueryRunner.java | 29 +- .../io/druid/query/ConcatQueryRunner.java | 4 +- .../DefaultGenericQueryMetricsFactory.java | 5 +- .../io/druid/query/DefaultQueryMetrics.java | 90 +- .../io/druid/query/DruidProcessingConfig.java | 1 + .../query/FinalizeResultsQueryRunner.java | 5 +- .../druid/query/FluentQueryRunnerBuilder.java | 4 +- .../query/GenericQueryMetricsFactory.java | 10 + .../druid/query/GroupByMergedQueryRunner.java | 12 +- .../query/IntervalChunkingQueryRunner.java | 15 +- .../query/MetricsEmittingExecutorService.java | 1 + .../query/MetricsEmittingQueryRunner.java | 25 +- .../java/io/druid/query/NoopQueryRunner.java | 2 +- .../src/main/java/io/druid/query/Query.java | 11 + .../java/io/druid/query/QueryContexts.java | 36 +- .../java/io/druid/query/QueryDataSource.java | 1 + .../java/io/druid/query/QueryMetrics.java | 10 +- .../main/java/io/druid/query/QueryPlus.java | 134 + .../main/java/io/druid/query/QueryRunner.java | 22 +- .../io/druid/query/QueryRunnerHelper.java | 4 +- .../ReferenceCountingSegmentQueryRunner.java | 6 +- ...portTimelineMissingSegmentQueryRunner.java | 2 +- .../query/ResourceLimitExceededException.java | 6 +- .../druid/query/ResultMergeQueryRunner.java | 5 +- .../java/io/druid/query/RetryQueryRunner.java | 10 +- .../io/druid/query/SubqueryQueryRunner.java | 8 +- .../java/io/druid/query/TableDataSource.java | 5 +- .../java/io/druid/query/TimewarpOperator.java | 33 +- .../java/io/druid/query/UnionQueryRunner.java | 7 +- .../druid/query/aggregation/Aggregator.java | 1 + .../query/aggregation/BufferAggregator.java | 4 +- .../aggregation/CountBufferAggregator.java | 1 + .../DoubleMaxAggregatorFactory.java | 3 +- .../DoubleMaxBufferAggregator.java | 16 +- .../DoubleMinAggregatorFactory.java | 3 +- .../DoubleMinBufferAggregator.java | 16 +- .../DoubleSumAggregatorFactory.java | 3 +- .../DoubleSumBufferAggregator.java | 12 +- .../aggregation/LongMaxAggregatorFactory.java | 3 +- .../aggregation/LongMinAggregatorFactory.java | 3 +- .../aggregation/LongSumAggregatorFactory.java | 3 +- .../aggregation/NoopBufferAggregator.java | 1 + ...java => SimpleDoubleBufferAggregator.java} | 33 +- .../cardinality/CardinalityAggregator.java | 4 +- .../hyperloglog/HyperUniquesAggregator.java | 3 +- .../HyperUniquesAggregatorFactory.java | 3 +- .../post/ArithmeticPostAggregator.java | 6 + .../post/ExpressionPostAggregator.java | 1 + .../post/JavaScriptPostAggregator.java | 1 + .../DataSourceMetadataQuery.java | 17 +- .../DataSourceMetadataQueryRunnerFactory.java | 10 +- .../DataSourceQueryQueryToolChest.java | 7 +- .../ForwardingFilteredDimensionSelector.java | 25 +- .../dimension/ListFilteredDimensionSpec.java | 5 +- .../PredicateFilteredDimensionSelector.java | 15 + .../dimension/RegexFilteredDimensionSpec.java | 2 +- .../query/extraction/CascadeExtractionFn.java | 11 +- .../extraction/JavaScriptExtractionFn.java | 1 + ...oatValueMatcherColumnSelectorStrategy.java | 14 + ...ongValueMatcherColumnSelectorStrategy.java | 14 + .../io/druid/query/filter/NotDimFilter.java | 1 + .../io/druid/query/filter/ValueMatcher.java | 8 +- .../groupby/DefaultGroupByQueryMetrics.java | 6 +- .../DefaultGroupByQueryMetricsFactory.java | 5 +- .../io/druid/query/groupby/GroupByQuery.java | 288 +- .../query/groupby/GroupByQueryConfig.java | 23 + .../query/groupby/GroupByQueryEngine.java | 1 + .../groupby/GroupByQueryMetricsFactory.java | 11 + .../groupby/GroupByQueryQueryToolChest.java | 18 +- .../groupby/GroupByQueryRunnerFactory.java | 19 +- .../RowBasedColumnSelectorFactory.java | 51 +- .../epinephelinae/AbstractBufferGrouper.java | 214 ++ .../groupby/epinephelinae/BufferGrouper.java | 435 +-- .../epinephelinae/ByteBufferHashTable.java | 382 ++ .../epinephelinae/ByteBufferIntList.java | 78 + .../ByteBufferMinMaxOffsetHeap.java | 493 +++ .../epinephelinae/ConcurrentGrouper.java | 17 +- .../GroupByMergingQueryRunnerV2.java | 19 +- .../epinephelinae/GroupByQueryEngineV2.java | 11 +- .../query/groupby/epinephelinae/Grouper.java | 22 +- .../query/groupby/epinephelinae/Groupers.java | 6 +- .../epinephelinae/LimitedBufferGrouper.java | 519 +++ .../epinephelinae/RowBasedGrouperHelper.java | 649 +++- .../epinephelinae/SpillingGrouper.java | 54 +- .../groupby/having/AlwaysHavingSpec.java | 8 - .../query/groupby/having/AndHavingSpec.java | 24 +- .../groupby/having/DimFilterHavingSpec.java | 14 +- .../having/DimensionSelectorHavingSpec.java | 65 +- .../groupby/having/EqualToHavingSpec.java | 23 +- .../groupby/having/GreaterThanHavingSpec.java | 23 +- .../query/groupby/having/HavingSpec.java | 3 +- .../groupby/having/LessThanHavingSpec.java | 23 +- .../query/groupby/having/NeverHavingSpec.java | 8 - .../query/groupby/having/NotHavingSpec.java | 14 +- .../query/groupby/having/OrHavingSpec.java | 24 +- .../groupby/orderby/DefaultLimitSpec.java | 33 +- .../groupby/orderby/OrderByColumnSpec.java | 46 + .../groupby/strategy/GroupByStrategyV1.java | 39 +- .../groupby/strategy/GroupByStrategyV2.java | 105 +- .../query/lookup/LookupReferencesManager.java | 7 +- .../metadata/SegmentMetadataQueryConfig.java | 22 + .../SegmentMetadataQueryQueryToolChest.java | 43 +- .../SegmentMetadataQueryRunnerFactory.java | 38 +- .../metadata/SegmentMetadataQuery.java | 64 +- .../monomorphicprocessing/HotLoopCallee.java | 2 +- .../SpecializationService.java | 38 +- .../query/ordering/StringComparators.java | 7 +- .../search/SearchQueryQueryToolChest.java | 15 +- .../druid/query/search/SearchQueryRunner.java | 4 +- .../query/search/search/SearchQuery.java | 16 +- .../query/search/search/SearchSortSpec.java | 1 + .../io/druid/query/select/PagingOffset.java | 6 + .../io/druid/query/select/SelectQuery.java | 22 +- .../select/SelectQueryQueryToolChest.java | 8 +- .../select/SelectQueryRunnerFactory.java | 4 +- .../druid/query/spec/LegacySegmentSpec.java | 3 +- .../druid/query/spec/QuerySegmentSpecs.java | 4 +- .../spec/SpecificSegmentQueryRunner.java | 8 +- .../druid/query/spec/SpecificSegmentSpec.java | 6 +- .../query/timeboundary/TimeBoundaryQuery.java | 33 +- .../TimeBoundaryQueryQueryToolChest.java | 11 +- .../TimeBoundaryQueryRunnerFactory.java | 5 +- .../DefaultTimeseriesQueryMetrics.java | 4 +- .../DefaultTimeseriesQueryMetricsFactory.java | 5 +- .../query/timeseries/TimeseriesQuery.java | 30 +- .../TimeseriesQueryMetricsFactory.java | 11 + .../TimeseriesQueryQueryToolChest.java | 12 +- .../TimeseriesQueryRunnerFactory.java | 4 +- .../query/topn/DefaultTopNQueryMetrics.java | 8 +- .../topn/DefaultTopNQueryMetricsFactory.java | 5 +- .../topn/Historical1AggPooledTopNScanner.java | 48 + ...leDoubleAggPooledTopNScannerPrototype.java | 74 + ...leDoubleAggPooledTopNScannerPrototype.java | 68 + .../druid/query/topn/PooledTopNAlgorithm.java | 197 +- .../topn/TopNLexicographicResultBuilder.java | 6 + .../query/topn/TopNNumericResultBuilder.java | 6 + .../java/io/druid/query/topn/TopNQuery.java | 42 +- .../query/topn/TopNQueryMetricsFactory.java | 11 + .../query/topn/TopNQueryQueryToolChest.java | 25 +- .../query/topn/TopNQueryRunnerFactory.java | 8 +- .../java/io/druid/query/topn/TopNUtils.java | 45 + .../java/io/druid/segment/BitmapOffset.java | 2 +- .../ColumnSelectorBitmapIndexSelector.java | 7 + .../CompressedVSizeIndexedSupplier.java | 15 + .../druid/segment/DimensionSelectorUtils.java | 32 + .../java/io/druid/segment/FilteredOffset.java | 206 ++ .../druid/segment/FloatDimensionIndexer.java | 2 + .../segment/FloatMetricColumnSerializer.java | 1 + .../main/java/io/druid/segment/IndexIO.java | 5 +- .../java/io/druid/segment/IndexMerger.java | 114 +- .../java/io/druid/segment/IndexMergerV9.java | 16 +- .../druid/segment/LongDimensionIndexer.java | 2 + .../segment/LongMetricColumnSerializer.java | 1 + .../java/io/druid/segment/MetricHolder.java | 21 - .../druid/segment/NullDimensionSelector.java | 22 +- .../java/io/druid/segment/QueryableIndex.java | 1 + .../QueryableIndexIndexableAdapter.java | 7 + .../segment/QueryableIndexStorageAdapter.java | 465 +-- .../segment/ReferenceCountingSegment.java | 1 + .../main/java/io/druid/segment/Rowboat.java | 17 +- .../segment/SingleScanTimeDimSelector.java | 21 +- .../segment/SingleValueDimensionSelector.java | 34 + .../druid/segment/StringDimensionIndexer.java | 59 +- .../segment/StringDimensionMergerV9.java | 12 +- .../java/io/druid/segment/VirtualColumns.java | 1 + .../segment/ZeroFloatColumnSelector.java | 10 +- .../druid/segment/ZeroLongColumnSelector.java | 1 + .../column/DictionaryEncodedColumn.java | 5 + .../column/SimpleDictionaryEncodedColumn.java | 192 + .../segment/data/ArrayBasedIndexedInts.java | 6 + .../druid/segment/data/ArrayBasedOffset.java | 3 +- .../io/druid/segment/data/ArrayIndexed.java | 8 + .../data/BitmapCompressedIndexedInts.java | 6 + .../data/BlockLayoutIndexedLongSupplier.java | 1 + .../BlockLayoutLongSupplierSerializer.java | 3 +- .../io/druid/segment/data/CachingIndexed.java | 12 +- .../data/CompressedIntsIndexedSupplier.java | 12 + .../data/CompressedIntsIndexedWriter.java | 4 +- .../CompressedVSizeIntsIndexedSupplier.java | 19 +- .../segment/data/CompressionFactory.java | 7 +- .../segment/data/DeltaLongEncodingWriter.java | 1 + .../druid/segment/data/EmptyIndexedInts.java | 6 + .../io/druid/segment/data/GenericIndexed.java | 37 +- .../segment/data/GenericIndexedWriter.java | 4 +- .../java/io/druid/segment/data/Indexed.java | 6 +- .../io/druid/segment/data/IndexedInts.java | 6 +- .../segment/data/IndexedIntsIterator.java | 5 +- .../druid/segment/data/IndexedMultivalue.java | 2 + .../IntermediateLongSupplierSerializer.java | 7 + .../segment/data/IntersectingOffset.java | 2 +- .../io/druid/segment/data/ListIndexed.java | 8 + .../java/io/druid/segment/data/Offset.java | 24 +- .../druid/segment/data/RangeIndexedInts.java | 6 + .../druid/segment/data/SingleIndexedInt.java | 7 + .../io/druid/segment/data/UnioningOffset.java | 2 +- .../io/druid/segment/data/VSizeIndexed.java | 7 + .../druid/segment/data/VSizeIndexedInts.java | 6 + .../segment/data/VSizeIndexedWriter.java | 12 +- .../io/druid/segment/data/VSizeLongSerde.java | 1 + .../druid/segment/data/ZeroIndexedInts.java | 6 + .../io/druid/segment/filter/AndFilter.java | 35 +- .../filter/ColumnComparisonFilter.java | 18 +- .../segment/filter/FalseValueMatcher.java | 7 + .../java/io/druid/segment/filter/Filters.java | 8 + .../io/druid/segment/filter/NotFilter.java | 7 + .../io/druid/segment/filter/OrFilter.java | 36 +- .../segment/filter/TrueValueMatcher.java | 7 + .../segment/historical/HistoricalCursor.java | 26 + .../HistoricalDimensionSelector.java | 33 + .../HistoricalFloatColumnSelector.java | 29 + .../segment/historical/OffsetHolder.java | 27 + ...ingleValueHistoricalDimensionSelector.java | 30 + .../segment/incremental/IncrementalIndex.java | 324 +- .../incremental/IncrementalIndexAdapter.java | 18 +- .../IncrementalIndexStorageAdapter.java | 29 +- .../incremental/OffheapIncrementalIndex.java | 14 +- .../incremental/OnheapIncrementalIndex.java | 10 +- .../SpatialDimensionRowTransformer.java | 4 +- .../DictionaryEncodedColumnPartSerde.java | 12 +- .../BaseSingleValueDimensionSelector.java | 23 +- .../segment/virtual/ExpressionSelectors.java | 1 + .../input/ProtoBufInputRowParserTest.java | 110 - .../data/input/ProtoTestEventWrapper.java | 1068 ------ .../granularity/QueryGranularityTest.java | 211 +- .../io/druid/guice/GuiceInjectorsTest.java | 2 +- .../guice/SegmentMetadataQueryConfigTest.java | 95 + .../io/druid/query/AsyncQueryRunnerTest.java | 6 +- .../ChainedExecutionQueryRunnerTest.java | 2 +- .../IntervalChunkingQueryRunnerTest.java | 6 +- .../druid/query/LegacyApiQueryRunnerTest.java | 64 + .../druid/query/MultiValuedDimensionTest.java | 4 +- .../io/druid/query/QueryContextsTest.java | 52 - .../io/druid/query/QueryRunnerTestHelper.java | 15 +- .../io/druid/query/RetryQueryRunnerTest.java | 15 +- .../test/java/io/druid/query/TestQuery.java | 75 + .../io/druid/query/TimewarpOperatorTest.java | 9 +- .../io/druid/query/UnionQueryRunnerTest.java | 8 +- .../aggregation/AggregationTestHelper.java | 7 +- .../aggregation/FilteredAggregatorTest.java | 1 + .../JavaScriptAggregatorBenchmark.java | 1 + .../aggregation/JavaScriptAggregatorTest.java | 4 +- .../CardinalityAggregatorBenchmark.java | 1 + .../CardinalityAggregatorTest.java | 6 + .../dimension/TestDimensionSelector.java | 1 + .../groupby/GroupByQueryMergeBufferTest.java | 1 + .../GroupByQueryRunnerFactoryTest.java | 16 +- .../GroupByQueryRunnerFailureTest.java | 6 +- .../query/groupby/GroupByQueryRunnerTest.java | 679 +++- .../druid/query/groupby/GroupByQueryTest.java | 33 +- .../GroupByTimeseriesQueryRunnerTest.java | 7 +- .../epinephelinae/BufferGrouperTest.java | 51 +- .../ByteBufferMinMaxOffsetHeapTest.java | 261 ++ .../epinephelinae/ConcurrentGrouperTest.java | 27 +- .../groupby/epinephelinae/IntKeySerde.java | 13 +- .../LimitedBufferGrouperTest.java | 174 + .../DimensionSelectorHavingSpecTest.java | 55 +- .../query/groupby/having/HavingSpecTest.java | 6 - .../LookupExtractorFactoryContainerTest.java | 1 + .../query/lookup/LookupExtractorTest.java | 6 +- ...egmentMetadataQueryQueryToolChestTest.java | 3 +- .../metadata/SegmentMetadataQueryTest.java | 62 +- .../StringRuntimeShapeTest.java | 1 + .../query/search/SearchQueryRunnerTest.java | 9 +- .../spec/SpecificSegmentQueryRunnerTest.java | 10 +- .../TimeSeriesUnionQueryRunnerTest.java | 6 +- .../topn/TopNQueryQueryToolChestTest.java | 6 +- .../druid/query/topn/TopNQueryRunnerTest.java | 202 +- .../CompressedVSizeIndexedV3SupplierTest.java | 2 + .../IndexMergerV9CompatibilityTest.java | 1 - .../druid/segment/SchemalessTestFullTest.java | 1 + .../segment/TestFloatColumnSelector.java | 4 +- .../test/java/io/druid/segment/TestIndex.java | 4 +- .../druid/segment/TestLongColumnSelector.java | 4 +- .../data/CompressedFloatsSerdeTest.java | 8 +- .../CompressedIntsIndexedSupplierTest.java | 3 +- .../data/CompressedLongsSerdeTest.java | 4 +- .../CompressedVSizeIndexedV3WriterTest.java | 12 +- ...ompressedVSizeIntsIndexedSupplierTest.java | 3 +- .../druid/segment/data/IndexedIntsTest.java | 3 +- .../segment/filter/FilterPartitionTest.java | 6 +- .../OnheapIncrementalIndexBenchmark.java | 6 +- .../OnheapIncrementalIndexTest.java | 5 +- .../segment/virtual/VirtualColumnsTest.java | 38 +- processing/src/test/resources/ProtoTest.proto | 31 - processing/src/test/resources/prototest.desc | Bin 400 -> 0 bytes .../test/resources/test.runtime.properties | 2 + server/pom.xml | 4 + .../AbstractCuratorServerInventoryView.java | 365 ++ .../client/BatchServerInventoryView.java | 3 +- .../druid/client/CachingClusteredClient.java | 19 +- .../io/druid/client/CachingQueryRunner.java | 8 +- .../druid/client/CoordinatorServerView.java | 12 + .../io/druid/client/DirectDruidClient.java | 204 +- .../java/io/druid/client/DruidServer.java | 26 +- .../io/druid/client/DruidServerDiscovery.java | 174 + ...lteredHttpServerInventoryViewProvider.java | 78 + .../FilteredServerInventoryViewProvider.java | 3 +- .../druid/client/HttpServerInventoryView.java | 663 ++++ .../client/HttpServerInventoryViewConfig.java | 61 + .../HttpServerInventoryViewProvider.java | 80 + .../client/ImmutableDruidDataSource.java | 11 + .../io/druid/client/ImmutableDruidServer.java | 14 +- .../java/io/druid/client/InventoryView.java | 8 +- .../io/druid/client/ServerInventoryView.java | 324 +- .../client/ServerInventoryViewProvider.java | 3 +- .../client/SingleServerInventoryView.java | 3 +- .../java/io/druid/client/cache/MapCache.java | 1 + .../io/druid/client/cache/MemcachedCache.java | 3 +- ...mcachedCustomConnectionFactoryBuilder.java | 1 + .../AbstractTierSelectorStrategy.java | 39 +- ...ConnectionCountServerSelectorStrategy.java | 14 + .../RandomServerSelectorStrategy.java | 21 +- .../druid/client/selector/ServerSelector.java | 80 +- .../selector/ServerSelectorStrategy.java | 5 +- .../client/selector/TierSelectorStrategy.java | 13 +- .../java/io/druid/curator/CuratorModule.java | 12 +- .../druid/curator/announcement/Announcer.java | 5 + .../inventory/CuratorInventoryManager.java | 22 +- .../java/io/druid/guice/AnnouncerModule.java | 5 +- .../java/io/druid/guice/FirehoseModule.java | 4 +- .../java/io/druid/guice/NodeTypeConfig.java | 8 +- .../java/io/druid/guice/ParsersModule.java | 2 - .../io/druid/guice/QueryToolChestModule.java | 57 +- .../java/io/druid/guice/ServerModule.java | 5 +- .../java/io/druid/guice/ServerViewModule.java | 2 + .../SQLMetadataStorageUpdaterJobHandler.java | 1 + .../IndexerSQLMetadataStorageCoordinator.java | 7 +- .../metadata/MetadataRuleManagerProvider.java | 1 + .../MetadataSegmentManagerProvider.java | 1 + .../MetadataSegmentPublisherProvider.java | 1 + .../druid/metadata/SQLMetadataConnector.java | 2 + .../metadata/SQLMetadataRuleManager.java | 7 + .../metadata/SQLMetadataSegmentManager.java | 2 + .../SQLMetadataStorageActionHandler.java | 11 + ...QLMetadataStorageActionHandlerFactory.java | 1 + .../SQLMetadataSupervisorManager.java | 1 + .../indexing/RealtimeTuningConfig.java | 12 +- .../granularity/UniformGranularitySpec.java | 1 + .../loading/LocalDataSegmentPusher.java | 19 +- .../SegmentLoaderLocalCacheManager.java | 8 +- .../segment/realtime/FireDepartment.java | 2 +- .../segment/realtime/RealtimeManager.java | 28 +- .../realtime/appenderator/Appenderator.java | 55 +- .../appenderator/AppenderatorConfig.java | 2 - .../appenderator/AppenderatorDriver.java | 673 ++++ .../AppenderatorDriverAddResult.java | 81 + ...a.java => AppenderatorDriverMetadata.java} | 16 +- .../appenderator/AppenderatorImpl.java | 98 +- .../appenderator/AppenderatorPlumber.java | 9 +- .../realtime/appenderator/Appenderators.java | 6 - .../FiniteAppenderatorDriver.java | 667 ---- .../appenderator/SegmentAllocator.java | 4 +- .../appenderator/SinkQuerySegmentWalker.java | 2 +- .../firehose/ClippedFirehoseFactory.java | 5 +- .../firehose/CombiningFirehoseFactory.java | 13 +- .../EventReceiverFirehoseFactory.java | 3 +- .../firehose/FixedCountFirehoseFactory.java | 5 +- .../firehose/HttpFirehoseFactory.java | 74 + .../realtime/firehose/IrcFirehoseFactory.java | 5 +- .../firehose/LocalFirehoseFactory.java | 76 +- .../firehose/ReplayableFirehoseFactory.java | 319 -- .../firehose/TimedShutoffFirehoseFactory.java | 11 +- ...oordinatorBasedSegmentHandoffNotifier.java | 2 +- .../realtime/plumber/RealtimePlumber.java | 6 +- .../plumber/SegmentHandoffNotifier.java | 1 + .../druid/segment/realtime/plumber/Sink.java | 2 +- .../java/io/druid/server/QueryManager.java | 1 + .../java/io/druid/server/QueryResource.java | 18 +- .../java/io/druid/server/SegmentManager.java | 266 ++ .../server/audit/AuditManagerProvider.java | 1 + .../BatchDataSegmentAnnouncer.java | 166 +- ...=> CuratorDataSegmentServerAnnouncer.java} | 34 +- .../coordination/DataSegmentAnnouncer.java | 5 - .../DataSegmentServerAnnouncer.java | 28 + .../coordination/DruidServerMetadata.java | 10 +- .../SegmentChangeRequestHistory.java | 354 ++ .../SegmentChangeRequestsSnapshot.java | 108 + .../server/coordination/ServerManager.java | 163 +- .../druid/server/coordination/ServerType.java | 91 + .../server/coordination/ZkCoordinator.java | 61 +- .../server/coordinator/CoordinatorStats.java | 94 +- .../coordinator/CostBalancerStrategy.java | 4 +- .../server/coordinator/DruidCluster.java | 99 +- .../server/coordinator/DruidCoordinator.java | 83 +- .../server/coordinator/LoadQueuePeon.java | 10 +- .../coordinator/SegmentReplicantLookup.java | 2 +- .../server/coordinator/ServerHolder.java | 13 +- .../helper/DruidCoordinatorBalancer.java | 2 +- .../DruidCoordinatorCleanupOvershadowed.java | 2 +- .../DruidCoordinatorCleanupUnneeded.java | 2 +- .../helper/DruidCoordinatorLogger.java | 341 +- .../helper/DruidCoordinatorSegmentMerger.java | 4 +- .../coordinator/rules/IntervalLoadRule.java | 1 + .../server/coordinator/rules/LoadRule.java | 9 +- .../server/http/CoordinatorRedirectInfo.java | 14 +- .../server/http/CoordinatorResource.java | 35 +- .../http/LookupCoordinatorResource.java | 316 ++ .../server/http/SegmentListerResource.java | 229 ++ .../io/druid/server/http/ServersResource.java | 4 +- .../http/security/StateResourceFilter.java | 1 + .../BatchDataSegmentAnnouncerConfig.java | 7 + .../server/initialization/ServerConfig.java | 10 + .../jetty/ResponseHeaderFilterHolder.java | 2 +- .../druid/server/log/FileRequestLogger.java | 4 +- .../cache/LookupCoordinatorManager.java | 19 +- .../metrics/HistoricalMetricsMonitor.java | 24 +- .../router/TieredBrokerHostSelector.java | 1 + .../timeline/partition/LinearShardSpec.java | 3 + .../timeline/partition/NumberedShardSpec.java | 7 + .../partition/SingleDimensionShardSpec.java | 1 + .../static/old-console/js/init-0.0.2.js | 8 +- .../io/druid/client/BrokerServerViewTest.java | 5 +- ...chingClusteredClientFunctionalityTest.java | 68 +- .../client/CachingClusteredClientTest.java | 41 +- .../druid/client/CachingQueryRunnerTest.java | 5 +- .../client/CoordinatorServerViewTest.java | 7 +- .../druid/client/DirectDruidClientTest.java | 61 +- .../client/BatchServerInventoryViewTest.java | 66 +- .../client/ImmutableSegmentLoadInfoTest.java | 3 +- .../client/selector/ServerSelectorTest.java | 17 +- .../selector/TierSelectorStrategyTest.java | 15 +- ...exerSQLMetadataStorageCoordinatorTest.java | 111 + .../LocatedSegmentDescriptorSerdeTest.java | 7 +- .../LookupIntrospectionResourceImplTest.java | 1 + .../CombiningFirehoseFactoryTest.java | 5 +- .../ReplayableFirehoseFactoryTest.java | 447 --- .../loading/LocalDataSegmentPusherTest.java | 6 +- .../segment/realtime/RealtimeManagerTest.java | 12 +- .../AppenderatorDriverFailTest.java | 464 +++ ...rTest.java => AppenderatorDriverTest.java} | 289 +- .../appenderator/AppenderatorTest.java | 32 +- .../appenderator/AppenderatorTester.java | 13 +- .../firehose/EventReceiverFirehoseTest.java | 7 +- .../firehose/IngestSegmentFirehoseTest.java | 161 +- .../firehose/LocalFirehoseFactoryTest.java | 110 + ...viceAnnouncingChatHandlerProviderTest.java | 2 +- ...inatorBasedSegmentHandoffNotifierTest.java | 7 +- .../AsyncQueryForwardingServletTest.java | 1 + .../druid/server/ClientInfoResourceTest.java | 9 +- .../io/druid/server/QueryResourceTest.java | 7 +- .../io/druid/server/SegmentManagerTest.java | 439 +++ .../SegmentChangeRequestHistoryTest.java | 246 ++ .../coordination/ServerManagerTest.java | 20 +- .../server/coordination/ServerTypeTest.java | 59 + .../coordination/ZkCoordinatorTest.java | 41 +- .../BatchDataSegmentAnnouncerTest.java | 58 +- .../coordinator/CoordinatorStatsTest.java | 136 + .../CostBalancerStrategyBenchmark.java | 7 +- .../coordinator/CostBalancerStrategyTest.java | 3 +- ...iskNormalizedCostBalancerStrategyTest.java | 3 +- .../server/coordinator/DruidClusterTest.java | 204 ++ .../DruidCoordinatorBalancerProfiler.java | 3 + .../DruidCoordinatorBalancerTest.java | 11 +- .../DruidCoordinatorBalancerTester.java | 3 +- .../DruidCoordinatorRuleRunnerTest.java | 141 +- .../coordinator/DruidCoordinatorTest.java | 38 +- .../coordinator/LoadQueuePeonTester.java | 1 + .../server/coordinator/ServerHolderTest.java | 246 ++ ...uidCoordinatorCleanupOvershadowedTest.java | 14 +- .../rules/BroadcastDistributionRuleTest.java | 26 +- .../coordinator/rules/LoadRuleTest.java | 31 +- .../http/CoordinatorRedirectInfoTest.java | 17 +- .../server/http/CoordinatorResourceTest.java | 76 + .../server/http/DatasourcesResourceTest.java | 5 +- .../server/http/IntervalsResourceTest.java | 3 +- .../http/LookupCoordinatorResourceTest.java | 233 ++ .../server/http/ServersResourceTest.java | 33 +- .../cache/LookupCoordinatorManagerTest.java | 15 +- .../metrics/HistoricalMetricsMonitorTest.java | 18 +- .../main/java/io/druid/cli/CliHistorical.java | 10 +- .../src/main/java/io/druid/cli/CliPeon.java | 5 +- .../java/io/druid/cli/CliRealtimeExample.java | 28 +- .../main/java/io/druid/cli/DumpSegment.java | 3 +- .../main/java/io/druid/cli/GuiceRunnable.java | 5 +- .../main/java/io/druid/cli/InsertSegment.java | 2 +- .../cli/Log4JShutdownPropertyChecker.java | 4 +- .../java/io/druid/guice/RealtimeModule.java | 13 +- .../sql/calcite/planner/PlannerContext.java | 7 + .../sql/calcite/planner/PlannerFactory.java | 8 +- .../io/druid/sql/calcite/rel/QueryMaker.java | 72 +- .../sql/calcite/rule/DruidSemiJoinRule.java | 2 + .../druid/sql/calcite/schema/DruidSchema.java | 24 +- .../calcite/view/InProcessViewManager.java | 3 + .../sql/avatica/DruidAvaticaHandlerTest.java | 58 +- .../druid/sql/avatica/DruidStatementTest.java | 3 +- .../druid/sql/calcite/CalciteQueryTest.java | 34 +- .../sql/calcite/http/SqlResourceTest.java | 6 +- .../sql/calcite/schema/DruidSchemaTest.java | 4 +- .../druid/sql/calcite/util/CalciteTests.java | 4 +- .../SpecificSegmentsQuerySegmentWalker.java | 6 +- .../calcite/util/TestServerInventoryView.java | 3 +- 829 files changed, 26385 insertions(+), 10066 deletions(-) create mode 100644 .idea/inspectionProfiles/Druid.xml create mode 100644 INTELLIJ_SETUP.md create mode 100644 api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java create mode 100644 api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java delete mode 100644 api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java create mode 100644 api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java delete mode 100644 api/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java create mode 100755 ci/travis_script_integration.sh create mode 100644 common/src/main/java/io/druid/annotations/SubclassesMustBePublic.java create mode 100644 common/src/main/java/io/druid/annotations/SubclassesMustBePublicAnnotationProcessor.java delete mode 100644 common/src/main/java/io/druid/collections/CountingMap.java create mode 100644 common/src/main/resources/services/javax.annotation.processing.Processor delete mode 100644 common/src/test/java/io/druid/collections/CountingMapTest.java create mode 100644 docs/content/development/extensions-core/protobuf.md create mode 100644 examples/quickstart/protobuf/kafka-metrics-pb.json create mode 100644 examples/quickstart/protobuf/metrics.desc create mode 100644 examples/quickstart/protobuf/metrics.proto create mode 100644 examples/quickstart/protobuf/metrics_pb2.py create mode 100755 examples/quickstart/protobuf/pb_publisher.py rename extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/{ExtractionNamespaceCacheFactory.java => CacheGenerator.java} (84%) rename extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/{JDBCExtractionNamespace.java => JdbcExtractionNamespace.java} (95%) rename extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/{URIExtractionNamespace.java => UriExtractionNamespace.java} (89%) rename extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/{JDBCExtractionNamespaceCacheFactory.java => JdbcCacheGenerator.java} (85%) rename extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/{StaticMapExtractionNamespaceCacheFactory.java => StaticMapCacheGenerator.java} (84%) rename extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/{URIExtractionNamespaceCacheFactory.java => UriCacheGenerator.java} (91%) rename extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/{URIExtractionNamespaceTest.java => UriExtractionNamespaceTest.java} (85%) rename extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/{StaticMapExtractionNamespaceCacheFactoryTest.java => StaticMapCacheGeneratorTest.java} (80%) rename extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/{URIExtractionNamespaceCacheFactoryTest.java => UriCacheGeneratorTest.java} (84%) rename extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/{NamespaceExtractionCacheManagerExecutorsTest.java => CacheSchedulerTest.java} (89%) rename extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/{JDBCExtractionNamespaceTest.java => JdbcExtractionNamespaceTest.java} (92%) create mode 100644 extensions-core/protobuf-extensions/pom.xml rename processing/src/main/java/io/druid/segment/data/ListBasedIndexedInts.java => extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java (55%) create mode 100644 extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java create mode 100755 extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java create mode 100644 extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java create mode 100644 extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto create mode 100644 extensions-core/protobuf-extensions/src/test/resources/prototest.desc delete mode 100644 java-util/src/main/java/io/druid/java/util/common/collect/CountingMap.java delete mode 100644 processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java create mode 100644 processing/src/main/java/io/druid/query/QueryPlus.java rename processing/src/main/java/io/druid/query/aggregation/{DoubleBufferAggregator.java => SimpleDoubleBufferAggregator.java} (61%) create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferIntList.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java create mode 100644 processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java create mode 100644 processing/src/main/java/io/druid/query/topn/Historical1AggPooledTopNScanner.java create mode 100644 processing/src/main/java/io/druid/query/topn/Historical1SimpleDoubleAggPooledTopNScannerPrototype.java create mode 100644 processing/src/main/java/io/druid/query/topn/HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNUtils.java create mode 100644 processing/src/main/java/io/druid/segment/FilteredOffset.java create mode 100644 processing/src/main/java/io/druid/segment/SingleValueDimensionSelector.java create mode 100644 processing/src/main/java/io/druid/segment/historical/HistoricalCursor.java create mode 100644 processing/src/main/java/io/druid/segment/historical/HistoricalDimensionSelector.java create mode 100644 processing/src/main/java/io/druid/segment/historical/HistoricalFloatColumnSelector.java create mode 100644 processing/src/main/java/io/druid/segment/historical/OffsetHolder.java create mode 100644 processing/src/main/java/io/druid/segment/historical/SingleValueHistoricalDimensionSelector.java delete mode 100644 processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java delete mode 100644 processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java create mode 100644 processing/src/test/java/io/druid/guice/SegmentMetadataQueryConfigTest.java create mode 100644 processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java create mode 100644 processing/src/test/java/io/druid/query/TestQuery.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeapTest.java create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java delete mode 100644 processing/src/test/resources/ProtoTest.proto delete mode 100644 processing/src/test/resources/prototest.desc create mode 100644 server/src/main/java/io/druid/client/AbstractCuratorServerInventoryView.java create mode 100644 server/src/main/java/io/druid/client/DruidServerDiscovery.java create mode 100644 server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java create mode 100644 server/src/main/java/io/druid/client/HttpServerInventoryView.java create mode 100644 server/src/main/java/io/druid/client/HttpServerInventoryViewConfig.java create mode 100644 server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java rename server/src/main/java/io/druid/segment/realtime/appenderator/{FiniteAppenderatorDriverMetadata.java => AppenderatorDriverMetadata.java} (78%) delete mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java create mode 100644 server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java delete mode 100644 server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java create mode 100644 server/src/main/java/io/druid/server/SegmentManager.java rename server/src/main/java/io/druid/server/coordination/{AbstractDataSegmentAnnouncer.java => CuratorDataSegmentServerAnnouncer.java} (77%) create mode 100644 server/src/main/java/io/druid/server/coordination/DataSegmentServerAnnouncer.java create mode 100644 server/src/main/java/io/druid/server/coordination/SegmentChangeRequestHistory.java create mode 100644 server/src/main/java/io/druid/server/coordination/SegmentChangeRequestsSnapshot.java create mode 100644 server/src/main/java/io/druid/server/coordination/ServerType.java create mode 100644 server/src/main/java/io/druid/server/http/SegmentListerResource.java delete mode 100644 server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java create mode 100644 server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java rename server/src/test/java/io/druid/segment/realtime/appenderator/{FiniteAppenderatorDriverTest.java => AppenderatorDriverTest.java} (51%) create mode 100644 server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java create mode 100644 server/src/test/java/io/druid/server/SegmentManagerTest.java create mode 100644 server/src/test/java/io/druid/server/coordination/SegmentChangeRequestHistoryTest.java create mode 100644 server/src/test/java/io/druid/server/coordination/ServerTypeTest.java create mode 100644 server/src/test/java/io/druid/server/coordinator/CoordinatorStatsTest.java create mode 100644 server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java create mode 100644 server/src/test/java/io/druid/server/coordinator/ServerHolderTest.java create mode 100644 server/src/test/java/io/druid/server/http/CoordinatorResourceTest.java diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml new file mode 100644 index 000000000000..c820649923e1 --- /dev/null +++ b/.idea/inspectionProfiles/Druid.xml @@ -0,0 +1,22 @@ + + + + \ No newline at end of file diff --git a/.travis.yml b/.travis.yml index 6d9068c56b8d..54fa4bcd8964 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,12 +3,29 @@ language: java jdk: - oraclejdk8 -script: - - mvn test -B -Pparallel-test -Dmaven.fork.count=2 && mvn clean -Pstrict compile test-compile -B - -sudo: false - cache: directories: - - $HOME/.m2 + - $HOME/.m2 +matrix: + include: + - sudo: false + install: true + script: + # The script execution consists of 4 steps: + # 1) Increase the jvm max heap size. Strict compilation in 2) requires more than 2 GB. + # 2) Strict compilation using error-prone + # 3) Reset the maven options. Parallel testing in 4) launches 2 processes, each requires 1 GB of memory. Increasing the maximum memory may cause OutOfMemory error because the instance of Travis has 4 GB of memory space. + # 4) Parallel unit testing + # Using && instead of independent script steps to make Travis build fail faster, if each step is not successful + - echo "MAVEN_OPTS='-Xmx3000m'" > ~/.mavenrc && mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B && rm ~/.mavenrc && mvn test -B -Pparallel-test -Dmaven.fork.count=2 + - sudo: required + services: + - docker + env: + - DOCKER_IP=172.17.0.1 + install: + # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. + - mvn clean install -q -ff -DskipTests -B + script: + - $TRAVIS_BUILD_DIR/ci/travis_script_integration.sh diff --git a/INTELLIJ_SETUP.md b/INTELLIJ_SETUP.md new file mode 100644 index 000000000000..384c2ce3897d --- /dev/null +++ b/INTELLIJ_SETUP.md @@ -0,0 +1,61 @@ +# IntelliJ Setup +This document contains some examples and instructions on how to get IntelliJ setup to run local debugging and test setups of Druid. + +## Code Style +The Code Style is available in XML format at [druid_intellij_formatting.xml](druid_intellij_formatting.xml) and can be [imported into IntelliJ](https://www.jetbrains.com/help/idea/2017.1/copying-code-style-settings.html). + +## Metadata +The installation of a MySQL metadata store is outside the scope of this document, but instructions on setting up MySQL can be found at [docs/content/development/extensions-core/mysql.md](docs/content/development/extensions-core/mysql.md). This assumes you followed the example there and have a database named `druid` with proper permissions for a user named `druid` and a password of `diurd`. + +Use of other databases such as Postgres or Derby are entirely reasonable, but doing so is left as an excercise to the reader. + +## ZooKeeper +This also assumes you have [ZooKeeper](http://zookeeper.apache.org/releases.html) running locally, which usually just involves downloading the latst distribution of ZooKeeper, doing some minor configuration in ZooKeeper's `conf/` directory (most defaults are fine), then running `./bin/zkServer.sh start` in the ZooKeeper directory. + +## Initial Build +Before running the apps, you should do a `mvn clean install -DskipTests` in the Druid source in order to make sure directories are populated correctly. + +## XML App Def +You can configure application definitions in XML for import into IntelliJ. Below are a few examples. These should be placed in an XML file in [.idea/runConfigurations](.idea/runConfigurations) in the Druid source code. + +### Historical.xml +```xml + + + + + +``` + +### Coordinator.xml +```xml + + + + + +``` diff --git a/api/src/main/java/io/druid/data/input/Firehose.java b/api/src/main/java/io/druid/data/input/Firehose.java index a768e778d81a..4f4c640f1040 100644 --- a/api/src/main/java/io/druid/data/input/Firehose.java +++ b/api/src/main/java/io/druid/data/input/Firehose.java @@ -19,6 +19,7 @@ package io.druid.data.input; +import javax.annotation.Nullable; import java.io.Closeable; /** @@ -46,14 +47,16 @@ public interface Firehose extends Closeable * * @return true if and when there is another row available, false if the stream has dried up */ - public boolean hasMore(); + boolean hasMore(); /** * The next row available. Should only be called if hasMore returns true. + * The return value can be null which means the caller must skip this row. * * @return The next row */ - public InputRow nextRow(); + @Nullable + InputRow nextRow(); /** * Returns a runnable that will "commit" everything read up to the point at which commit() is called. This is @@ -74,5 +77,5 @@ public interface Firehose extends Closeable * because of InputRows delivered by prior calls to ##nextRow(). *

*/ - public Runnable commit(); + Runnable commit(); } diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java index 43cbf01111f9..2494c13ea716 100644 --- a/api/src/main/java/io/druid/data/input/FirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java @@ -20,12 +20,18 @@ package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonTypeInfo; - import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.parsers.ParseException; +import java.io.File; import java.io.IOException; +/** + * FirehoseFactory creates a {@link Firehose} which is an interface holding onto the stream of incoming data. + * It currently provides two methods for creating a {@link Firehose} and their default implementations call each other + * for the backward compatibility. Implementations of this interface must implement one of these methods. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface FirehoseFactory { @@ -36,7 +42,31 @@ public interface FirehoseFactory * If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on * invalid configuration is preferred over returning null. + * + * @param parser an input row parser */ - public Firehose connect(T parser) throws IOException, ParseException; + @Deprecated + default Firehose connect(T parser) throws IOException, ParseException + { + return connect(parser, null); + } + /** + * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to + * call hasMore() on the returned Firehose (which might subsequently block). + *

+ * If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return + * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on + * invalid configuration is preferred over returning null. + *

+ * Some fire hoses like {@link PrefetchableTextFilesFirehoseFactory} may use a temporary + * directory to cache data in it. + * + * @param parser an input row parser + * @param temporaryDirectory a directory where temporary files are stored + */ + default Firehose connect(T parser, File temporaryDirectory) throws IOException, ParseException + { + return connect(parser); + } } diff --git a/api/src/main/java/io/druid/data/input/MapBasedRow.java b/api/src/main/java/io/druid/data/input/MapBasedRow.java index 77e4fd9f3d38..534c5eec9290 100644 --- a/api/src/main/java/io/druid/data/input/MapBasedRow.java +++ b/api/src/main/java/io/druid/data/input/MapBasedRow.java @@ -76,6 +76,7 @@ public long getTimestampFromEpoch() return timestamp.getMillis(); } + @Override @JsonProperty public DateTime getTimestamp() { diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java new file mode 100644 index 000000000000..a8428b8fd299 --- /dev/null +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -0,0 +1,124 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.java.util.common.logger.Logger; +import org.apache.commons.io.Charsets; +import org.apache.commons.io.IOUtils; +import org.apache.commons.io.LineIterator; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * This is an abstract class for firehose factory for making firehoses reading text files. + * It provides an unified {@link #connect(StringInputRowParser, File)} implementation for its subclasses. + * + * @param object type representing input data + */ +public abstract class AbstractTextFilesFirehoseFactory + implements FirehoseFactory +{ + private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); + + private List objects; + + @Override + public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException + { + if (objects == null) { + objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); + } + final Iterator iterator = objects.iterator(); + return new FileIteratingFirehose( + new Iterator() + { + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public LineIterator next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final ObjectType object = iterator.next(); + try { + return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), Charsets.UTF_8); + } + catch (Exception e) { + LOG.error( + e, + "Exception reading object[%s]", + object + ); + throw Throwables.propagate(e); + } + } + }, + firehoseParser + ); + } + + /** + * Initialize objects to be read by this firehose. Since firehose factories are constructed whenever + * io.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred + * until {@link #connect(StringInputRowParser, File)} is called. + * + * @return a collection of initialized objects. + */ + protected abstract Collection initObjects() throws IOException; + + /** + * Open an input stream from the given object. If the object is compressed, this method should return a byte stream + * as it is compressed. The object compression should be handled in {@link #wrapObjectStream(Object, InputStream)}. + * + * @param object an object to be read + * + * @return an input stream for the object + * + * @throws IOException + */ + protected abstract InputStream openObjectStream(ObjectType object) throws IOException; + + /** + * Wrap the given input stream if needed. The decompression logic should be applied to the given stream if the object + * is compressed. + * + * @param object an input object + * @param stream a stream for the object + * @return + * @throws IOException + */ + protected abstract InputStream wrapObjectStream(ObjectType object, InputStream stream) throws IOException; +} diff --git a/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java b/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java index bbe1fc4d2288..a4b09a2010a9 100644 --- a/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; - import io.druid.java.util.common.parsers.CSVParser; import io.druid.java.util.common.parsers.Parser; @@ -35,26 +34,38 @@ public class CSVParseSpec extends ParseSpec { private final String listDelimiter; private final List columns; + private final boolean hasHeaderRow; + private final int skipHeaderRows; @JsonCreator public CSVParseSpec( @JsonProperty("timestampSpec") TimestampSpec timestampSpec, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, @JsonProperty("listDelimiter") String listDelimiter, - @JsonProperty("columns") List columns + @JsonProperty("columns") List columns, + @JsonProperty("hasHeaderRow") boolean hasHeaderRow, + @JsonProperty("skipHeaderRows") int skipHeaderRows ) { super(timestampSpec, dimensionsSpec); this.listDelimiter = listDelimiter; - Preconditions.checkNotNull(columns, "columns"); - for (String column : columns) { - Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); - } - this.columns = columns; - - verify(dimensionsSpec.getDimensionNames()); + this.hasHeaderRow = hasHeaderRow; + this.skipHeaderRows = skipHeaderRows; + + if (columns != null) { + for (String column : columns) { + Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); + } + verify(dimensionsSpec.getDimensionNames()); + } else { + Preconditions.checkArgument( + hasHeaderRow, + "If columns field is not set, the first row of your data must have your header" + + " and hasHeaderRow must be set to true." + ); + } } @JsonProperty @@ -69,6 +80,18 @@ public List getColumns() return columns; } + @JsonProperty + public boolean isHasHeaderRow() + { + return hasHeaderRow; + } + + @JsonProperty("skipHeaderRows") + public int getSkipHeaderRows() + { + return skipHeaderRows; + } + @Override public void verify(List usedCols) { @@ -80,23 +103,23 @@ public void verify(List usedCols) @Override public Parser makeParser() { - return new CSVParser(Optional.fromNullable(listDelimiter), columns); + return new CSVParser(Optional.fromNullable(listDelimiter), columns, hasHeaderRow, skipHeaderRows); } @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { - return new CSVParseSpec(spec, getDimensionsSpec(), listDelimiter, columns); + return new CSVParseSpec(spec, getDimensionsSpec(), listDelimiter, columns, hasHeaderRow, skipHeaderRows); } @Override public ParseSpec withDimensionsSpec(DimensionsSpec spec) { - return new CSVParseSpec(getTimestampSpec(), spec, listDelimiter, columns); + return new CSVParseSpec(getTimestampSpec(), spec, listDelimiter, columns, hasHeaderRow, skipHeaderRows); } public ParseSpec withColumns(List cols) { - return new CSVParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols); + return new CSVParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols, hasHeaderRow, skipHeaderRows); } } diff --git a/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java b/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java index 6d7096d7921f..c3383eb351c4 100644 --- a/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; - import io.druid.java.util.common.parsers.DelimitedParser; import io.druid.java.util.common.parsers.Parser; @@ -36,6 +35,8 @@ public class DelimitedParseSpec extends ParseSpec private final String delimiter; private final String listDelimiter; private final List columns; + private final boolean hasHeaderRow; + private final int skipHeaderRows; @JsonCreator public DelimitedParseSpec( @@ -43,20 +44,31 @@ public DelimitedParseSpec( @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, @JsonProperty("delimiter") String delimiter, @JsonProperty("listDelimiter") String listDelimiter, - @JsonProperty("columns") List columns + @JsonProperty("columns") List columns, + @JsonProperty("hasHeaderRow") boolean hasHeaderRow, + @JsonProperty("skipHeaderRows") int skipHeaderRows ) { super(timestampSpec, dimensionsSpec); this.delimiter = delimiter; this.listDelimiter = listDelimiter; - Preconditions.checkNotNull(columns, "columns"); this.columns = columns; - for (String column : this.columns) { - Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); + this.hasHeaderRow = hasHeaderRow; + this.skipHeaderRows = skipHeaderRows; + + if (columns != null) { + for (String column : this.columns) { + Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); + } + verify(dimensionsSpec.getDimensionNames()); + } else { + Preconditions.checkArgument( + hasHeaderRow, + "If columns field is not set, the first row of your data must have your header" + + " and hasHeaderRow must be set to true." + ); } - - verify(dimensionsSpec.getDimensionNames()); } @JsonProperty("delimiter") @@ -77,6 +89,18 @@ public List getColumns() return columns; } + @JsonProperty + public boolean isHasHeaderRow() + { + return hasHeaderRow; + } + + @JsonProperty("skipHeaderRows") + public int getSkipHeaderRows() + { + return skipHeaderRows; + } + @Override public void verify(List usedCols) { @@ -88,38 +112,79 @@ public void verify(List usedCols) @Override public Parser makeParser() { - Parser retVal = new DelimitedParser( + return new DelimitedParser( Optional.fromNullable(delimiter), - Optional.fromNullable(listDelimiter) + Optional.fromNullable(listDelimiter), + columns, + hasHeaderRow, + skipHeaderRows ); - retVal.setFieldNames(columns); - return retVal; } @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { - return new DelimitedParseSpec(spec, getDimensionsSpec(), delimiter, listDelimiter, columns); + return new DelimitedParseSpec( + spec, + getDimensionsSpec(), + delimiter, + listDelimiter, + columns, + hasHeaderRow, + skipHeaderRows + ); } @Override public ParseSpec withDimensionsSpec(DimensionsSpec spec) { - return new DelimitedParseSpec(getTimestampSpec(), spec, delimiter, listDelimiter, columns); + return new DelimitedParseSpec( + getTimestampSpec(), + spec, + delimiter, + listDelimiter, + columns, + hasHeaderRow, + skipHeaderRows + ); } public ParseSpec withDelimiter(String delim) { - return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delim, listDelimiter, columns); + return new DelimitedParseSpec( + getTimestampSpec(), + getDimensionsSpec(), + delim, + listDelimiter, + columns, + hasHeaderRow, + skipHeaderRows + ); } public ParseSpec withListDelimiter(String delim) { - return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, delim, columns); + return new DelimitedParseSpec( + getTimestampSpec(), + getDimensionsSpec(), + delimiter, + delim, + columns, + hasHeaderRow, + skipHeaderRows + ); } public ParseSpec withColumns(List cols) { - return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, listDelimiter, cols); + return new DelimitedParseSpec( + getTimestampSpec(), + getDimensionsSpec(), + delimiter, + listDelimiter, + cols, + hasHeaderRow, + skipHeaderRows + ); } } diff --git a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java index 97e33f04a894..d1d0606ea49e 100644 --- a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java +++ b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java @@ -19,14 +19,15 @@ package io.druid.data.input.impl; -import com.google.common.base.Throwables; import io.druid.data.input.Firehose; import io.druid.data.input.InputRow; import io.druid.utils.Runnables; import org.apache.commons.io.LineIterator; +import java.io.Closeable; import java.io.IOException; import java.util.Iterator; +import java.util.NoSuchElementException; /** */ @@ -37,20 +38,32 @@ public class FileIteratingFirehose implements Firehose private LineIterator lineIterator = null; + private final Closeable closer; + public FileIteratingFirehose( Iterator lineIterators, StringInputRowParser parser ) + { + this(lineIterators, parser, null); + } + + public FileIteratingFirehose( + Iterator lineIterators, + StringInputRowParser parser, + Closeable closer + ) { this.lineIterators = lineIterators; this.parser = parser; + this.closer = closer; } @Override public boolean hasMore() { while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) { - lineIterator = lineIterators.next(); + lineIterator = getNextLineIterator(); } return lineIterator != null && lineIterator.hasNext(); @@ -59,21 +72,22 @@ public boolean hasMore() @Override public InputRow nextRow() { - try { - if (lineIterator == null || !lineIterator.hasNext()) { - // Close old streams, maybe. - if (lineIterator != null) { - lineIterator.close(); - } + if (!hasMore()) { + throw new NoSuchElementException(); + } - lineIterator = lineIterators.next(); - } + return parser.parse(lineIterator.next()); + } - return parser.parse(lineIterator.next()); - } - catch (Exception e) { - throw Throwables.propagate(e); + private LineIterator getNextLineIterator() + { + if (lineIterator != null) { + lineIterator.close(); } + + final LineIterator iterator = lineIterators.next(); + parser.startFileFromBeginning(); + return iterator; } @Override @@ -85,8 +99,24 @@ public Runnable commit() @Override public void close() throws IOException { - if (lineIterator != null) { - lineIterator.close(); + try { + if (lineIterator != null) { + lineIterator.close(); + } + } + catch (Throwable t) { + try { + if (closer != null) { + closer.close(); + } + } + catch (Exception e) { + t.addSuppressed(e); + } + throw t; + } + if (closer != null) { + closer.close(); } } } diff --git a/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java new file mode 100644 index 000000000000..b5bc768870fd --- /dev/null +++ b/api/src/main/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactory.java @@ -0,0 +1,537 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.io.CountingOutputStream; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.druid.data.input.Firehose; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import org.apache.commons.io.Charsets; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.commons.io.LineIterator; + +import java.io.Closeable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; + +/** + * PrefetchableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned + * by this class provides three key functionalities. + * + *

    + *
  • + * Caching: for the first call of {@link #connect(StringInputRowParser, File)}, it caches objects in a local disk + * up to {@link #maxCacheCapacityBytes}. These caches are NOT deleted until the process terminates, + * and thus can be used for future reads. + *
  • + *
  • + * Fetching: when it reads all cached data, it fetches remaining objects into a local disk and reads data from + * them. For the performance reason, prefetch technique is used, that is, when the size of remaining cached or + * fetched data is smaller than {@link #prefetchTriggerBytes}, a background prefetch thread automatically starts to + * fetch remaining objects. + *
  • + *
  • + * Retry: if an exception occurs while downloading an object, it retries again up to {@link #maxFetchRetry}. + *
  • + *
+ * + * This implementation can be useful when the cost for reading input objects is large as reading from AWS S3 because + * IndexTask can read the whole data twice for determining partition specs and generating segments if the intervals of + * GranularitySpec is not specified. + * + * Prefetching can be turned on/off by setting {@link #maxFetchCapacityBytes}. Depending on prefetching is enabled or + * disabled, the behavior of the firehose is different like below. + * + *
    + *
  1. + * If prefetch is enabled, PrefetchableTextFilesFirehose can fetch input objects in background. + *
  2. + *
  3. When next() is called, it first checks that there are already fetched files in local storage. + *
      + *
    1. + * If exists, it simply chooses a fetched file and returns a {@link LineIterator} reading that file. + *
    2. + *
    3. + * If there is no fetched files in local storage but some objects are still remained to be read, the firehose + * fetches one of input objects in background immediately. If an IOException occurs while downloading the object, + * it retries up to the maximum retry count. Finally, the firehose returns a {@link LineIterator} only when the + * download operation is successfully finished. + *
    4. + *
    + *
  4. + *
  5. + * If prefetch is disabled, the firehose returns a {@link LineIterator} which directly reads the stream opened by + * {@link #openObjectStream}. If there is an IOException, it will throw it and the read will fail. + *
  6. + *
+ */ +public abstract class PrefetchableTextFilesFirehoseFactory + extends AbstractTextFilesFirehoseFactory +{ + private static final Logger LOG = new Logger(PrefetchableTextFilesFirehoseFactory.class); + private static final long DEFAULT_MAX_CACHE_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB + private static final long DEFAULT_MAX_FETCH_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GB + private static final long DEFAULT_FETCH_TIMEOUT = 60_000; // 60 secs + private static final int DEFAULT_MAX_FETCH_RETRY = 3; + private static final String FETCH_FILE_PREFIX = "fetch-"; + + // The below two variables are roughly the max size of total cached/fetched objects, but the actual cached/fetched + // size can be larger. The reason is our current client implementations for cloud storages like s3 don't support range + // scan yet, so we must download the whole file at once. It's still possible for the size of cached/fetched data to + // not exceed these variables by estimating the after-fetch size, but it makes us consider the case when any files + // cannot be fetched due to their large size, which makes the implementation complicated. + private final long maxCacheCapacityBytes; + private final long maxFetchCapacityBytes; + + private final long prefetchTriggerBytes; + + // timeout for fetching an object from the remote site + private final long fetchTimeout; + + // maximum retry for fetching an object from the remote site + private final int maxFetchRetry; + + private final List cacheFiles = new ArrayList<>(); + private long totalCachedBytes; + + private List objects; + + private static ExecutorService createFetchExecutor() + { + return Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setNameFormat("firehose_fetch_%d") + .build() + ); + } + + public PrefetchableTextFilesFirehoseFactory( + Long maxCacheCapacityBytes, + Long maxFetchCapacityBytes, + Long prefetchTriggerBytes, + Long fetchTimeout, + Integer maxFetchRetry + ) + { + this.maxCacheCapacityBytes = maxCacheCapacityBytes == null + ? DEFAULT_MAX_CACHE_CAPACITY_BYTES + : maxCacheCapacityBytes; + this.maxFetchCapacityBytes = maxFetchCapacityBytes == null + ? DEFAULT_MAX_FETCH_CAPACITY_BYTES + : maxFetchCapacityBytes; + this.prefetchTriggerBytes = prefetchTriggerBytes == null + ? this.maxFetchCapacityBytes / 2 + : prefetchTriggerBytes; + this.fetchTimeout = fetchTimeout == null ? DEFAULT_FETCH_TIMEOUT : fetchTimeout; + this.maxFetchRetry = maxFetchRetry == null ? DEFAULT_MAX_FETCH_RETRY : maxFetchRetry; + } + + @Override + public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException + { + if (maxCacheCapacityBytes == 0 && maxFetchCapacityBytes == 0) { + return super.connect(firehoseParser, temporaryDirectory); + } + + if (objects == null) { + objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); + } + + Preconditions.checkState(temporaryDirectory.exists(), "temporaryDirectory[%s] does not exist", temporaryDirectory); + Preconditions.checkState( + temporaryDirectory.isDirectory(), + "temporaryDirectory[%s] is not a directory", + temporaryDirectory + ); + + // fetchExecutor is responsible for background data fetching + final ExecutorService fetchExecutor = createFetchExecutor(); + + return new FileIteratingFirehose( + new Iterator() + { + // When prefetching is enabled, fetchFiles and nextFetchIndex are updated by the fetchExecutor thread, but + // read by both the main thread (in hasNext()) and the fetchExecutor thread (in fetch()). To guarantee that + // fetchFiles and nextFetchIndex are updated atomically, this lock must be held before updating + // them. + private final Object fetchLock = new Object(); + private final LinkedBlockingQueue fetchFiles = new LinkedBlockingQueue<>(); + + // Number of bytes currently fetched files. + // This is updated when a file is successfully fetched or a fetched file is deleted. + private final AtomicLong fetchedBytes = new AtomicLong(0); + private final boolean cacheInitialized; + private final boolean prefetchEnabled; + + private Future fetchFuture; + private int cacheIterateIndex; + // nextFetchIndex indicates which object should be downloaded when fetch is triggered. + private int nextFetchIndex; + + { + cacheInitialized = totalCachedBytes > 0; + prefetchEnabled = maxFetchCapacityBytes > 0; + + if (cacheInitialized) { + nextFetchIndex = cacheFiles.size(); + } + if (prefetchEnabled) { + fetchIfNeeded(totalCachedBytes); + } + } + + private void fetchIfNeeded(long remainingBytes) + { + if ((fetchFuture == null || fetchFuture.isDone()) + && remainingBytes <= prefetchTriggerBytes) { + fetchFuture = fetchExecutor.submit( + () -> { + fetch(); + return null; + } + ); + } + } + + /** + * Fetch objects to a local disk up to {@link PrefetchableTextFilesFirehoseFactory#maxFetchCapacityBytes}. + * This method is not thread safe and must be called by a single thread. Note that even + * {@link PrefetchableTextFilesFirehoseFactory#maxFetchCapacityBytes} is 0, at least 1 file is always fetched. + * This is for simplifying design, and should be improved when our client implementations for cloud storages + * like S3 support range scan. + */ + private void fetch() throws Exception + { + for (int i = nextFetchIndex; i < objects.size() && fetchedBytes.get() <= maxFetchCapacityBytes; i++) { + final ObjectType object = objects.get(i); + LOG.info("Fetching object[%s], fetchedBytes[%d]", object, fetchedBytes.get()); + final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory); + fetchedBytes.addAndGet(download(object, outFile, 0)); + synchronized (fetchLock) { + fetchFiles.put(new FetchedFile(object, outFile)); + nextFetchIndex++; + } + } + } + + /** + * Downloads an object. It retries downloading {@link PrefetchableTextFilesFirehoseFactory#maxFetchRetry} + * times and throws an exception. + * + * @param object an object to be downloaded + * @param outFile a file which the object data is stored + * @param tryCount current retry count + * + * @return number of downloaded bytes + * + * @throws IOException + */ + private long download(ObjectType object, File outFile, int tryCount) throws IOException + { + try (final InputStream is = openObjectStream(object); + final CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(outFile))) { + IOUtils.copy(is, cos); + return cos.getCount(); + } + catch (IOException e) { + final int nextTry = tryCount + 1; + if (!Thread.currentThread().isInterrupted() && nextTry < maxFetchRetry) { + LOG.error(e, "Failed to download object[%s], retrying (%d of %d)", object, nextTry, maxFetchRetry); + outFile.delete(); + return download(object, outFile, nextTry); + } else { + LOG.error(e, "Failed to download object[%s], retries exhausted, aborting", object); + throw e; + } + } + } + + @Override + public boolean hasNext() + { + synchronized (fetchLock) { + return (cacheInitialized && cacheIterateIndex < cacheFiles.size()) + || !fetchFiles.isEmpty() + || nextFetchIndex < objects.size(); + } + } + + @Override + public LineIterator next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + // If fetch() fails, hasNext() always returns true because nextFetchIndex must be smaller than the number + // of objects, which means next() is always called. The below method checks that fetch() threw an exception + // and propagates it if exists. + checkFetchException(); + + final OpenedObject openedObject; + + try { + // Check cache first + if (cacheInitialized && cacheIterateIndex < cacheFiles.size()) { + final FetchedFile fetchedFile = cacheFiles.get(cacheIterateIndex++); + openedObject = new OpenedObject(fetchedFile, getNoopCloser()); + } else if (prefetchEnabled) { + openedObject = openObjectFromLocal(); + } else { + openedObject = openObjectFromRemote(); + } + + final InputStream stream = wrapObjectStream( + openedObject.object, + openedObject.objectStream + ); + + return new ResourceCloseableLineIterator( + new InputStreamReader(stream, Charsets.UTF_8), + openedObject.resourceCloser + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private void checkFetchException() + { + if (fetchFuture != null && fetchFuture.isDone()) { + try { + fetchFuture.get(); + fetchFuture = null; + } + catch (InterruptedException | ExecutionException e) { + throw Throwables.propagate(e); + } + } + } + + private OpenedObject openObjectFromLocal() throws IOException + { + final FetchedFile fetchedFile; + final Closeable resourceCloser; + + if (!fetchFiles.isEmpty()) { + // If there are already fetched files, use them + fetchedFile = fetchFiles.poll(); + resourceCloser = cacheIfPossibleAndGetCloser(fetchedFile, fetchedBytes); + fetchIfNeeded(fetchedBytes.get()); + } else { + // Otherwise, wait for fetching + try { + fetchIfNeeded(fetchedBytes.get()); + fetchedFile = fetchFiles.poll(fetchTimeout, TimeUnit.MILLISECONDS); + if (fetchedFile == null) { + // Check the latest fetch is failed + checkFetchException(); + // Or throw a timeout exception + throw new RuntimeException(new TimeoutException()); + } + resourceCloser = cacheIfPossibleAndGetCloser(fetchedFile, fetchedBytes); + // trigger fetch again for subsequent next() calls + fetchIfNeeded(fetchedBytes.get()); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + return new OpenedObject(fetchedFile, resourceCloser); + } + + private OpenedObject openObjectFromRemote() throws IOException + { + final OpenedObject openedObject; + final Closeable resourceCloser = getNoopCloser(); + + if (totalCachedBytes < maxCacheCapacityBytes) { + LOG.info("Caching object[%s]", objects.get(nextFetchIndex)); + try { + // Since maxFetchCapacityBytes is 0, at most one file is fetched. + fetch(); + FetchedFile fetchedFile = fetchFiles.poll(); + if (fetchedFile == null) { + throw new ISE("Cannot fetch object[%s]", objects.get(nextFetchIndex)); + } + cacheIfPossible(fetchedFile); + fetchedBytes.addAndGet(-fetchedFile.length()); + openedObject = new OpenedObject(fetchedFile, resourceCloser); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } else { + final ObjectType object = objects.get(nextFetchIndex++); + LOG.info("Reading object[%s]", object); + openedObject = new OpenedObject(object, openObjectStream(object), resourceCloser); + } + return openedObject; + } + }, + firehoseParser, + () -> { + fetchExecutor.shutdownNow(); + try { + Preconditions.checkState(fetchExecutor.awaitTermination(fetchTimeout, TimeUnit.MILLISECONDS)); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown fetch executor during close"); + } + } + ); + } + + private boolean cacheIfPossible(FetchedFile fetchedFile) + { + // maxCacheCapacityBytes is a rough limit, so if totalCachedBytes is larger than it, no more caching is + // allowed. + if (totalCachedBytes < maxCacheCapacityBytes) { + cacheFiles.add(fetchedFile); + totalCachedBytes += fetchedFile.length(); + return true; + } else { + return false; + } + } + + private Closeable cacheIfPossibleAndGetCloser(FetchedFile fetchedFile, AtomicLong fetchedBytes) + { + final Closeable closeable; + if (cacheIfPossible(fetchedFile)) { + closeable = getNoopCloser(); + // If the fetchedFile is cached, make a room for fetching more data immediately. + // This is because cache space and fetch space are separated. + fetchedBytes.addAndGet(-fetchedFile.length()); + } else { + closeable = getFetchedFileCloser(fetchedFile, fetchedBytes); + } + return closeable; + } + + private Closeable getNoopCloser() + { + return () -> {}; + } + + private Closeable getFetchedFileCloser( + final FetchedFile fetchedFile, + final AtomicLong fetchedBytes + ) + { + return () -> { + final long fileSize = fetchedFile.length(); + fetchedFile.delete(); + fetchedBytes.addAndGet(-fileSize); + }; + } + + /** + * This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed. + */ + static class ResourceCloseableLineIterator extends LineIterator + { + private final Closeable resourceCloser; + + public ResourceCloseableLineIterator(Reader reader, Closeable resourceCloser) throws IllegalArgumentException + { + super(reader); + this.resourceCloser = resourceCloser; + } + + @Override + public void close() + { + super.close(); + try { + resourceCloser.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + + private class FetchedFile + { + private final ObjectType object; + private final File file; + + public FetchedFile(ObjectType object, File file) + { + this.object = object; + this.file = file; + } + + public long length() + { + return file.length(); + } + + public void delete() + { + file.delete(); + } + } + + private class OpenedObject + { + private final ObjectType object; + private final InputStream objectStream; + private final Closeable resourceCloser; + + public OpenedObject(FetchedFile fetchedFile, Closeable resourceCloser) throws IOException + { + this.object = fetchedFile.object; + this.objectStream = FileUtils.openInputStream(fetchedFile.file); + this.resourceCloser = resourceCloser; + } + + public OpenedObject(ObjectType object, InputStream objectStream, Closeable resourceCloser) + { + this.object = object; + this.objectStream = objectStream; + this.resourceCloser = resourceCloser; + } + } +} diff --git a/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java index 6a13fcb7bd19..a640ef10ac41 100644 --- a/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java +++ b/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java @@ -22,12 +22,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; - import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.InputRow; import io.druid.java.util.common.parsers.ParseException; import io.druid.java.util.common.parsers.Parser; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.CharBuffer; import java.nio.charset.Charset; @@ -124,18 +124,30 @@ private Map buildStringKeyMap(ByteBuffer input) return theMap; } - private Map parseString(String inputString) + public void startFileFromBeginning() { - return parser.parse(inputString); + parser.startFileFromBeginning(); } - public InputRow parse(String input) + @Nullable + public InputRow parse(@Nullable String input) { return parseMap(parseString(input)); } - private InputRow parseMap(Map theMap) + @Nullable + private Map parseString(@Nullable String inputString) { + return parser.parse(inputString); + } + + @Nullable + private InputRow parseMap(@Nullable Map theMap) + { + // If a header is present in the data (and with proper configurations), a null is returned + if (theMap == null) { + return null; + } return mapParser.parse(theMap); } } diff --git a/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java b/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java index e6740cb63f4b..9f81a18fc578 100644 --- a/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import io.druid.java.util.common.parsers.Parser; import java.util.List; @@ -43,6 +42,7 @@ public TimeAndDimsParseSpec( ); } + @Override public Parser makeParser() { return new Parser() @@ -67,11 +67,13 @@ public List getFieldNames() }; } + @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { return new TimeAndDimsParseSpec(spec, getDimensionsSpec()); } + @Override public ParseSpec withDimensionsSpec(DimensionsSpec spec) { return new TimeAndDimsParseSpec(getTimestampSpec(), spec); diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java b/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java index f77aa198c8a6..9638ed48b15c 100644 --- a/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java +++ b/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java @@ -19,15 +19,45 @@ package io.druid.segment.loading; +import com.google.common.base.Joiner; import io.druid.timeline.DataSegment; import java.io.File; import java.io.IOException; +import java.net.URI; +import java.util.Map; public interface DataSegmentPusher { + Joiner JOINER = Joiner.on("/").skipNulls(); + @Deprecated String getPathForHadoop(String dataSource); String getPathForHadoop(); DataSegment push(File file, DataSegment segment) throws IOException; + //use map instead of LoadSpec class to avoid dependency pollution. + Map makeLoadSpec(URI finalIndexZipFilePath); + default String getStorageDir(DataSegment dataSegment) { + return getDefaultStorageDir(dataSegment); + } + default String makeIndexPathName(DataSegment dataSegment, String indexName) { + return String.format("./%s/%s", getStorageDir(dataSegment),indexName); + } + + // Note: storage directory structure format = .../dataSource/interval/version/partitionNumber/ + // If above format is ever changed, make sure to change it appropriately in other places + // e.g. HDFSDataSegmentKiller uses this information to clean the version, interval and dataSource directories + // on segment deletion if segment being deleted was the only segment + static String getDefaultStorageDir(DataSegment segment) { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + } } diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java b/api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java deleted file mode 100644 index 7daa125088d9..000000000000 --- a/api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.loading; - -import com.google.common.base.Joiner; -import io.druid.timeline.DataSegment; -import org.joda.time.format.ISODateTimeFormat; - -/** - */ -public class DataSegmentPusherUtil -{ - private static final Joiner JOINER = Joiner.on("/").skipNulls(); - - // Note: storage directory structure format = .../dataSource/interval/version/partitionNumber/ - // If above format is ever changed, make sure to change it appropriately in other places - // e.g. HDFSDataSegmentKiller uses this information to clean the version, interval and dataSource directories - // on segment deletion if segment being deleted was the only segment - public static String getStorageDir(DataSegment segment) - { - return JOINER.join( - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() - ); - } - - /** - * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in - * path names. So we format paths differently for HDFS. - */ - public static String getHdfsStorageDir(DataSegment segment) - { - return JOINER.join( - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), - segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) - ), - segment.getVersion().replaceAll(":", "_") - ); - } -} diff --git a/api/src/main/java/io/druid/utils/Runnables.java b/api/src/main/java/io/druid/utils/Runnables.java index 793c9c1c3d05..a70ff66de484 100644 --- a/api/src/main/java/io/druid/utils/Runnables.java +++ b/api/src/main/java/io/druid/utils/Runnables.java @@ -25,6 +25,7 @@ public class Runnables { public static Runnable getNoopRunnable(){ return new Runnable(){ + @Override public void run(){} }; } diff --git a/api/src/test/java/io/druid/data/input/impl/CSVParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/CSVParseSpecTest.java index 7d99a2b804de..b4c70f31ddf9 100644 --- a/api/src/test/java/io/druid/data/input/impl/CSVParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/CSVParseSpecTest.java @@ -23,6 +23,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; public class CSVParseSpecTest { @@ -41,7 +42,9 @@ public void testColumnMissing() throws Exception Lists.newArrayList() ), ",", - Arrays.asList("a") + Collections.singletonList("a"), + false, + 0 ); } @@ -60,7 +63,9 @@ public void testComma() throws Exception Lists.newArrayList() ), ",", - Arrays.asList("a") + Collections.singletonList("a"), + false, + 0 ); } } diff --git a/api/src/test/java/io/druid/data/input/impl/DelimitedParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/DelimitedParseSpecTest.java index 1ebad8c64141..90428bb0f0ac 100644 --- a/api/src/test/java/io/druid/data/input/impl/DelimitedParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/DelimitedParseSpecTest.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; public class DelimitedParseSpecTest { @@ -37,10 +38,12 @@ public void testSerde() throws IOException { DelimitedParseSpec spec = new DelimitedParseSpec( new TimestampSpec("abc", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "\u0001", "\u0002", - Arrays.asList("abc") + Collections.singletonList("abc"), + false, + 0 ); final DelimitedParseSpec serde = jsonMapper.readValue( jsonMapper.writeValueAsString(spec), @@ -49,10 +52,10 @@ public void testSerde() throws IOException Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn()); Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); - Assert.assertEquals(Arrays.asList("abc"), serde.getColumns()); + Assert.assertEquals(Collections.singletonList("abc"), serde.getColumns()); Assert.assertEquals("\u0001", serde.getDelimiter()); Assert.assertEquals("\u0002", serde.getListDelimiter()); - Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames()); + Assert.assertEquals(Collections.singletonList("abc"), serde.getDimensionsSpec().getDimensionNames()); } @Test(expected = IllegalArgumentException.class) @@ -71,7 +74,9 @@ public void testColumnMissing() throws Exception ), ",", " ", - Arrays.asList("a") + Collections.singletonList("a"), + false, + 0 ); } @@ -91,12 +96,15 @@ public void testComma() throws Exception ), ",", null, - Arrays.asList("a") + Collections.singletonList("a"), + false, + 0 ); } - @Test(expected = NullPointerException.class) - public void testDefaultColumnList(){ + @Test(expected = IllegalArgumentException.class) + public void testDefaultColumnList() + { final DelimitedParseSpec spec = new DelimitedParseSpec( new TimestampSpec( "timestamp", @@ -110,8 +118,9 @@ public void testDefaultColumnList(){ ), ",", null, - // pass null columns not allowed - null + null, + false, + 0 ); } } diff --git a/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java b/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java index 87f5d20fcd5c..7b60959901fe 100644 --- a/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java +++ b/api/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java @@ -19,67 +19,152 @@ package io.druid.data.input.impl; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - -import io.druid.java.util.common.Pair; -import junit.framework.Assert; - +import io.druid.data.input.InputRow; import org.apache.commons.io.LineIterator; +import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; +import java.io.Closeable; +import java.io.IOException; +import java.io.Reader; import java.io.StringReader; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +@RunWith(Parameterized.class) public class FileIteratingFirehoseTest { - private static final List>> fixtures = ImmutableList.of( - Pair.of(new String[]{"2000,foo"}, ImmutableList.of("foo")), - Pair.of(new String[]{"2000,foo\n2000,bar\n"}, ImmutableList.of("foo", "bar")), - Pair.of(new String[]{"2000,foo\n2000,bar\n", "2000,baz"}, ImmutableList.of("foo", "bar", "baz")), - Pair.of(new String[]{"2000,foo\n2000,bar\n", "", "2000,baz"}, ImmutableList.of("foo", "bar", "baz")), - Pair.of(new String[]{"2000,foo\n2000,bar\n", "", "2000,baz", ""}, ImmutableList.of("foo", "bar", "baz")), - Pair.of(new String[]{""}, ImmutableList.of()), - Pair.of(new String[]{}, ImmutableList.of()) - ); + @Parameters(name = "{0}, {1}") + public static Collection constructorFeeder() throws IOException + { + final List> inputTexts = ImmutableList.of( + ImmutableList.of("2000,foo"), + ImmutableList.of("2000,foo\n2000,bar\n"), + ImmutableList.of("2000,foo\n2000,bar\n", "2000,baz"), + ImmutableList.of("2000,foo\n2000,bar\n", "", "2000,baz"), + ImmutableList.of("2000,foo\n2000,bar\n", "", "2000,baz", ""), + ImmutableList.of("2000,foo\n2000,bar\n2000,baz", "", "2000,baz", "2000,foo\n2000,bar\n3000,baz"), + ImmutableList.of(""), + ImmutableList.of() + ); + + final List args = new ArrayList<>(); + for (int numSkipHeadRows = 0; numSkipHeadRows < 3; numSkipHeadRows++) { + for (List texts : inputTexts) { + args.add(new Object[] { texts, numSkipHeadRows }); + } + } + + return args; + } + + private static final char[] LINE_CHARS = "\n".toCharArray(); + + private final StringInputRowParser parser; + private final List inputs; + private final List expectedResults; + + public FileIteratingFirehoseTest(List texts, int numSkipHeaderRows) + { + parser = new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null), + ",", + ImmutableList.of("ts", "x"), + false, + numSkipHeaderRows + ), + null + ); + + this.inputs = texts; + this.expectedResults = inputs.stream() + .map(input -> input.split("\n")) + .flatMap(lines -> { + final List filteredLines = Arrays.asList(lines).stream() + .filter(line -> line.length() > 0) + .map(line -> line.split(",")[1]) + .collect(Collectors.toList()); + + final int numRealSkippedRows = Math.min(filteredLines.size(), numSkipHeaderRows); + IntStream.range(0, numRealSkippedRows).forEach(i -> filteredLines.set(i, null)); + return filteredLines.stream(); + }) + .collect(Collectors.toList()); + } @Test public void testFirehose() throws Exception { - for (Pair> fixture : fixtures) { - final List lineIterators = Lists.transform( - Arrays.asList(fixture.lhs), - new Function() - { - @Override - public LineIterator apply(String s) - { - return new LineIterator(new StringReader(s)); - } - } - ); - - final StringInputRowParser parser = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null), - ",", - ImmutableList.of("ts", "x") - ), - null - ); - - final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser); + final List lineIterators = inputs.stream() + .map(s -> new LineIterator(new StringReader(s))) + .collect(Collectors.toList()); + + try (final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser)) { final List results = Lists.newArrayList(); while (firehose.hasMore()) { - results.add(Joiner.on("|").join(firehose.nextRow().getDimension("x"))); + final InputRow inputRow = firehose.nextRow(); + if (inputRow == null) { + results.add(null); + } else { + results.add(Joiner.on("|").join(inputRow.getDimension("x"))); + } + } + + Assert.assertEquals(expectedResults, results); + } + } + + @Test(expected = RuntimeException.class) + public void testClose() throws IOException + { + final LineIterator lineIterator = new LineIterator(new Reader() + { + @Override + public int read(char[] cbuf, int off, int len) throws IOException + { + System.arraycopy(LINE_CHARS, 0, cbuf, 0, LINE_CHARS.length); + return LINE_CHARS.length; + } + + @Override + public void close() throws IOException + { + throw new RuntimeException("close test for FileIteratingFirehose"); } + }); + + final TestCloseable closeable = new TestCloseable(); + final FileIteratingFirehose firehose = new FileIteratingFirehose( + ImmutableList.of(lineIterator).iterator(), + parser, + closeable + ); + firehose.hasMore(); // initialize lineIterator + firehose.close(); + Assert.assertTrue(closeable.closed); + } + + private static final class TestCloseable implements Closeable + { + private boolean closed; - Assert.assertEquals(fixture.rhs, results); + @Override + public void close() throws IOException + { + closed = true; } } } diff --git a/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java index 0a806b613562..e9dcc196f3a0 100644 --- a/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; - import io.druid.TestObjectMapper; import io.druid.java.util.common.parsers.Parser; import io.druid.js.JavaScriptConfig; @@ -32,7 +31,7 @@ import org.junit.rules.ExpectedException; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; import java.util.Map; /** @@ -55,7 +54,7 @@ public void testSerde() throws IOException ); JavaScriptParseSpec spec = new JavaScriptParseSpec( new TimestampSpec("abc", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "abc", JavaScriptConfig.getEnabledInstance() ); @@ -67,7 +66,7 @@ public void testSerde() throws IOException Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); Assert.assertEquals("abc", serde.getFunction()); - Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames()); + Assert.assertEquals(Collections.singletonList("abc"), serde.getDimensionsSpec().getDimensionNames()); } @Test @@ -76,7 +75,7 @@ public void testMakeParser() final JavaScriptConfig config = JavaScriptConfig.getEnabledInstance(); JavaScriptParseSpec spec = new JavaScriptParseSpec( new TimestampSpec("abc", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "function(str) { var parts = str.split(\"-\"); return { one: parts[0], two: parts[1] } }", config ); @@ -92,7 +91,7 @@ public void testMakeParserNotAllowed() final JavaScriptConfig config = new JavaScriptConfig(false); JavaScriptParseSpec spec = new JavaScriptParseSpec( new TimestampSpec("abc", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "abc", config ); diff --git a/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java index 4b38453fe35f..a1e95c028ada 100644 --- a/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/ParseSpecTest.java @@ -26,6 +26,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; public class ParseSpecTest { @@ -45,7 +46,9 @@ public void testDuplicateNames() throws Exception ), ",", " ", - Arrays.asList("a", "b") + Arrays.asList("a", "b"), + false, + 0 ); } @@ -65,7 +68,9 @@ public void testDimAndDimExcluOverlap() throws Exception ), ",", null, - Arrays.asList("a", "B") + Arrays.asList("a", "B"), + false, + 0 ); } @@ -79,13 +84,15 @@ public void testDimExclusionDuplicate() throws Exception null ), new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("a")), + DimensionsSpec.getDefaultSchemas(Collections.singletonList("a")), Lists.newArrayList("B", "B"), Lists.newArrayList() ), ",", null, - Arrays.asList("a", "B") + Arrays.asList("a", "B"), + false, + 0 ); } } diff --git a/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java new file mode 100644 index 000000000000..8fefb396ecf9 --- /dev/null +++ b/api/src/test/java/io/druid/data/input/impl/PrefetchableTextFilesFirehoseFactoryTest.java @@ -0,0 +1,420 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl; + +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import io.druid.data.input.Firehose; +import io.druid.data.input.Row; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.TrueFileFilter; +import org.hamcrest.CoreMatchers; +import org.joda.time.DateTime; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.Writer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +public class PrefetchableTextFilesFirehoseFactoryTest +{ + private static File testDir; + private static File firehoseTempDir; + + private static final StringInputRowParser parser = new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec( + "timestamp", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")), + Lists.newArrayList(), + Lists.newArrayList() + ), + ",", + Arrays.asList("timestamp", "a", "b"), + false, + 0 + ), + Charsets.UTF_8.name() + ); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @BeforeClass + public static void setup() throws IOException + { + testDir = File.createTempFile(PrefetchableTextFilesFirehoseFactoryTest.class.getSimpleName(), "testDir"); + FileUtils.forceDelete(testDir); + FileUtils.forceMkdir(testDir); + + firehoseTempDir = File.createTempFile(PrefetchableTextFilesFirehoseFactoryTest.class.getSimpleName(), "baseDir"); + FileUtils.forceDelete(firehoseTempDir); + FileUtils.forceMkdir(firehoseTempDir); + + for (int i = 0; i < 10; i++) { + // Each file is 1390 bytes + try (final Writer writer = new BufferedWriter( + new FileWriter(new File(testDir, "test_" + i)) + )) { + for (int j = 0; j < 100; j++) { + final String a = (20171220 + i) + "," + i + "," + j + "\n"; + writer.write(a); + } + } + } + } + + @AfterClass + public static void teardown() throws IOException + { + FileUtils.forceDelete(testDir); + FileUtils.forceDelete(firehoseTempDir); + } + + private static void assertResult(List rows) + { + Assert.assertEquals(1000, rows.size()); + rows.sort((r1, r2) -> { + int c = r1.getTimestamp().compareTo(r2.getTimestamp()); + if (c != 0) { + return c; + } + c = Integer.valueOf(r1.getDimension("a").get(0)).compareTo(Integer.valueOf(r2.getDimension("a").get(0))); + if (c != 0) { + return c; + } + + return Integer.valueOf(r1.getDimension("b").get(0)).compareTo(Integer.valueOf(r2.getDimension("b").get(0))); + }); + + for (int i = 0; i < 10; i++) { + for (int j = 0; j < 100; j++) { + final Row row = rows.get(i * 100 + j); + Assert.assertEquals(new DateTime(20171220 + i), row.getTimestamp()); + Assert.assertEquals(String.valueOf(i), row.getDimension("a").get(0)); + Assert.assertEquals(String.valueOf(j), row.getDimension("b").get(0)); + } + } + } + + @Test + public void testWithoutCacheAndFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 0, 0); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithoutCache() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 0, 2048); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithZeroFetchCapacity() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 2048, 0); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithCacheAndFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.of(testDir); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithLargeCacheAndSmallFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 2048, 1024); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testWithSmallCacheAndLargeFetch() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.with(testDir, 1024, 2048); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testRetry() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(testDir, 1); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + + assertResult(rows); + } + + @Test + public void testMaxRetry() throws IOException + { + expectedException.expect(RuntimeException.class); + expectedException.expectCause(CoreMatchers.instanceOf(ExecutionException.class)); + expectedException.expectMessage("Exception for retry test"); + + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(testDir, 5); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + } + + @Test + public void testTimeout() throws IOException + { + expectedException.expect(RuntimeException.class); + expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class)); + + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.withSleepMillis(testDir, 1000); + + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + } + + @Test + public void testReconnect() throws IOException + { + final TestPrefetchableTextFilesFirehoseFactory factory = + TestPrefetchableTextFilesFirehoseFactory.of(testDir); + + for (int i = 0; i < 5; i++) { + final List rows = new ArrayList<>(); + try (Firehose firehose = factory.connect(parser, firehoseTempDir)) { + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + } + assertResult(rows); + } + } + + static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory + { + private static final long defaultTimeout = 1000; + private final long sleepMillis; + private final File baseDir; + private int openExceptionCount; + + static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCapacity, long fetchCapacity) + { + return new TestPrefetchableTextFilesFirehoseFactory( + baseDir, + 1024, + cacheCapacity, + fetchCapacity, + defaultTimeout, + 3, + 0, + 0 + ); + } + + static TestPrefetchableTextFilesFirehoseFactory of(File baseDir) + { + return new TestPrefetchableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 2048, + defaultTimeout, + 3, + 0, + 0 + ); + } + + static TestPrefetchableTextFilesFirehoseFactory withOpenExceptions(File baseDir, int count) + { + return new TestPrefetchableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 2048, + defaultTimeout, + 3, + count, + 0 + ); + } + + static TestPrefetchableTextFilesFirehoseFactory withSleepMillis(File baseDir, long ms) + { + return new TestPrefetchableTextFilesFirehoseFactory( + baseDir, + 1024, + 2048, + 2048, + 100, + 3, + 0, + ms + ); + } + + public TestPrefetchableTextFilesFirehoseFactory( + File baseDir, + long prefetchTriggerThreshold, + long maxCacheCapacityBytes, + long maxFetchCapacityBytes, + long timeout, + int maxRetry, + int openExceptionCount, + long sleepMillis + ) + { + super( + maxCacheCapacityBytes, + maxFetchCapacityBytes, + prefetchTriggerThreshold, + timeout, + maxRetry + ); + this.openExceptionCount = openExceptionCount; + this.sleepMillis = sleepMillis; + this.baseDir = baseDir; + } + + @Override + protected Collection initObjects() + { + return FileUtils.listFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), + TrueFileFilter.INSTANCE, + TrueFileFilter.INSTANCE + ); + } + + @Override + protected InputStream openObjectStream(File object) throws IOException + { + if (openExceptionCount > 0) { + openExceptionCount--; + throw new IOException("Exception for retry test"); + } + if (sleepMillis > 0) { + try { + Thread.sleep(sleepMillis); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return FileUtils.openInputStream(object); + } + + @Override + protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException + { + return stream; + } + } +} diff --git a/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java b/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java index d3b86ee8f1df..68930ea6269d 100644 --- a/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java +++ b/api/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java @@ -25,7 +25,7 @@ import org.junit.Test; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; /** */ @@ -38,9 +38,9 @@ public void testSerde() throws IOException { RegexParseSpec spec = new RegexParseSpec( new TimestampSpec("abc", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "\u0001", - Arrays.asList("abc"), + Collections.singletonList("abc"), "abc" ); final RegexParseSpec serde = jsonMapper.readValue( @@ -52,6 +52,6 @@ public void testSerde() throws IOException Assert.assertEquals("abc", serde.getPattern()); Assert.assertEquals("\u0001", serde.getListDelimiter()); - Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames()); + Assert.assertEquals(Collections.singletonList("abc"), serde.getDimensionsSpec().getDimensionNames()); } } diff --git a/api/src/test/java/io/druid/guice/PolyBindTest.java b/api/src/test/java/io/druid/guice/PolyBindTest.java index eaa2583563a8..ef8806cd1705 100644 --- a/api/src/test/java/io/druid/guice/PolyBindTest.java +++ b/api/src/test/java/io/druid/guice/PolyBindTest.java @@ -32,6 +32,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Properties; /** @@ -46,7 +47,7 @@ public void setUp(Module... modules) throws Exception props = new Properties(); injector = Guice.createInjector( Iterables.concat( - Arrays.asList( + Collections.singletonList( new Module() { @Override diff --git a/api/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java b/api/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java deleted file mode 100644 index d2f1c6eab400..000000000000 --- a/api/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.loading; - -import com.google.common.collect.ImmutableMap; -import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.NoneShardSpec; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; - -public class DataSegmentPusherUtilTest -{ - @Test - public void shouldNotHaveColonsInHdfsStorageDir() throws Exception - { - - Interval interval = new Interval("2011-10-01/2011-10-02"); - ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); - - DataSegment segment = new DataSegment( - "something", - interval, - "brand:new:version", - loadSpec, - Arrays.asList("dim1", "dim2"), - Arrays.asList("met1", "met2"), - NoneShardSpec.instance(), - null, - 1 - ); - - String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment); - Assert.assertEquals("something/20111001T000000.000Z_20111002T000000.000Z/brand_new_version", storageDir); - - } -} diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java index 62de8735200e..5a15f55e8148 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java @@ -560,7 +560,7 @@ public List apply(Cursor input) ); } - private class NoBitmapSelectorFilter extends SelectorFilter + private static class NoBitmapSelectorFilter extends SelectorFilter { public NoBitmapSelectorFilter( String dimension, @@ -577,7 +577,7 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector) } } - private class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter + private static class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter { public NoBitmapDimensionPredicateFilter( final String dimension, @@ -595,7 +595,7 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector) } } - private class NoBitmapSelectorDimFilter extends SelectorDimFilter + private static class NoBitmapSelectorDimFilter extends SelectorDimFilter { public NoBitmapSelectorDimFilter( String dimension, diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java index 4c2537fd9dee..f065e725b754 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java @@ -95,6 +95,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -128,7 +129,11 @@ public class FilteredAggregatorBenchmark private File tmpDir; private static String JS_FN = "function(str) { return 'super-' + str; }"; - private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn(JS_FN, false, JavaScriptConfig.getEnabledInstance()); + private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn( + JS_FN, + false, + JavaScriptConfig.getEnabledInstance() + ); static { JSON_MAPPER = new DefaultObjectMapper(); @@ -169,10 +174,15 @@ public void setup() throws IOException filter = new OrDimFilter( Arrays.asList( new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC), - new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getEnabledInstance()), + new JavaScriptDimFilter( + "dimSequential", + "function(x) { return false }", + null, + JavaScriptConfig.getEnabledInstance() + ), new RegexDimFilter("dimSequential", "X", null), new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null), - new InDimFilter("dimSequential", Arrays.asList("X"), null) + new InDimFilter("dimSequential", Collections.singletonList("X"), null) ) ); filteredMetrics = new AggregatorFactory[1]; @@ -208,7 +218,7 @@ public void setup() throws IOException ); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(filteredMetrics[0]); diff --git a/benchmarks/src/main/java/io/druid/benchmark/FlattenJSONBenchmarkUtil.java b/benchmarks/src/main/java/io/druid/benchmark/FlattenJSONBenchmarkUtil.java index 00fbcdc97b85..0f7143f66d3d 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FlattenJSONBenchmarkUtil.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FlattenJSONBenchmarkUtil.java @@ -285,7 +285,7 @@ public String generateNestedEvent() throws Exception return mapper.writeValueAsString(wrapper); } - public class BenchmarkEvent + public static class BenchmarkEvent { public String ts; diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java index d737508eaff4..78211112449e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -96,7 +96,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -174,7 +174,7 @@ private void setupQueries() BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); { // basic.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "sumLongSequential", @@ -240,7 +240,7 @@ private void setupQueries() } { // basic.nested - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "sumLongSequential", diff --git a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java index de8f74111c3f..9ef2eb93a3ec 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -89,7 +89,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -159,7 +159,7 @@ private void setupQueries() BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); { // basic.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); @@ -199,7 +199,7 @@ private void setupQueries() basicQueries.put("float", queryBuilderFloat); } { // basic.numericSort - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); @@ -215,7 +215,7 @@ private void setupQueries() basicQueries.put("numericSort", queryBuilderA); } { // basic.alphanumericSort - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java index 966021b483bd..b41672a1e748 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java @@ -208,7 +208,7 @@ private void initDistribution() ((AbstractIntegerDistribution) distribution).reseedRandomGenerator(seed); } else if (distribution instanceof AbstractRealDistribution) { ((AbstractRealDistribution) distribution).reseedRandomGenerator(seed); - } else if (distribution instanceof EnumeratedDistribution) { + } else { ((EnumeratedDistribution) distribution).reseedRandomGenerator(seed); } } diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/EnumeratedTreeDistribution.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/EnumeratedTreeDistribution.java index 5cab5af7ee7e..d8ff5c9d136c 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/EnumeratedTreeDistribution.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/EnumeratedTreeDistribution.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.TreeMap; -/* +/** * EnumeratedDistrubtion's sample() method does a linear scan through the array of probabilities. * * This is too slow with high cardinality value sets, so this subclass overrides sample() to use diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/RealRoundingDistribution.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/RealRoundingDistribution.java index e913f3a8a0fe..04c78a486c65 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/RealRoundingDistribution.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/RealRoundingDistribution.java @@ -22,7 +22,7 @@ import org.apache.commons.math3.distribution.AbstractIntegerDistribution; import org.apache.commons.math3.distribution.AbstractRealDistribution; -/* +/** * Rounds the output values from the sample() function of an AbstractRealDistribution. */ public class RealRoundingDistribution extends AbstractIntegerDistribution diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/SequentialDistribution.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/SequentialDistribution.java index b73d6253e52f..dbabd9aabcae 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/SequentialDistribution.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/SequentialDistribution.java @@ -22,7 +22,7 @@ import org.apache.commons.math3.distribution.EnumeratedDistribution; import org.apache.commons.math3.util.Pair; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class SequentialDistribution extends EnumeratedDistribution @@ -37,7 +37,7 @@ public class SequentialDistribution extends EnumeratedDistribution public SequentialDistribution(Integer start, Integer end, List enumeratedValues) { // just pass in some bogus probability mass function, we won't be using it - super(Arrays.asList(new Pair(null, 1.0))); + super(Collections.singletonList(new Pair(null, 1.0))); this.start = start; this.end = end; this.enumeratedValues = enumeratedValues; diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 9e54a79bcd1b..a0b67105e01d 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -67,6 +67,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -173,7 +174,7 @@ public void readWithFilters(Blackhole blackhole) throws Exception new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getEnabledInstance()), new RegexDimFilter("dimSequential", "X", null), new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null), - new InDimFilter("dimSequential", Arrays.asList("X"), null) + new InDimFilter("dimSequential", Collections.singletonList("X"), null) ) ); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java index 0a1001e8a7c2..ded7ebba0a2b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -98,7 +98,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -174,7 +174,7 @@ private void setupQueries() BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); { // basic.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "sumLongSequential", @@ -202,7 +202,7 @@ private void setupQueries() } { // basic.nested - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "sumLongSequential", @@ -246,7 +246,7 @@ private void setupQueries() BenchmarkSchemaInfo simpleSchema = BenchmarkSchemas.SCHEMA_MAP.get("simple"); { // simple.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "rows", @@ -273,7 +273,7 @@ private void setupQueries() Map simpleLongQueries = new LinkedHashMap<>(); BenchmarkSchemaInfo simpleLongSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleLong"); { // simpleLong.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleLongSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleLongSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "rows", @@ -300,7 +300,7 @@ private void setupQueries() Map simpleFloatQueries = new LinkedHashMap<>(); BenchmarkSchemaInfo simpleFloatSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleFloat"); { // simpleFloat.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleFloatSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleFloatSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory( "rows", diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/QueryBenchmarkUtil.java b/benchmarks/src/main/java/io/druid/benchmark/query/QueryBenchmarkUtil.java index 662b0ed71e8e..a06bb2ba9757 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/QueryBenchmarkUtil.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/QueryBenchmarkUtil.java @@ -25,6 +25,7 @@ import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -58,9 +59,9 @@ public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) { return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - return delegate.run(query, responseContext); + return delegate.run(queryPlus, responseContext); } }; } diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java index b2584fc95454..a97d34fab585 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java @@ -97,7 +97,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -188,7 +188,7 @@ private static SearchQueryBuilder makeQuery(final String name, final BenchmarkSc private static SearchQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); return Druids.newSearchQueryBuilder() .dataSource("blah") @@ -199,7 +199,7 @@ private static SearchQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) private static SearchQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); final List dimUniformFilterVals = Lists.newArrayList(); int resultNum = (int) (100000 * 0.1); @@ -230,7 +230,7 @@ private static SearchQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) private static SearchQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); final List dimUniformFilterVals = Lists.newArrayList(); final int resultNum = (int) (100000 * 0.1); @@ -284,7 +284,7 @@ public ExtractionType getExtractionType() private static SearchQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); final List dimUniformFilterVals = Lists.newArrayList(); final int resultNum = (int) (100000 * 0.1); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java index 21033adc7a72..ef843f68c25a 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java @@ -88,6 +88,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -156,7 +157,7 @@ private void setupQueries() BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); { // basic.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); Druids.SelectQueryBuilder queryBuilderA = Druids.newSelectQueryBuilder() diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SerializingQueryRunner.java b/benchmarks/src/main/java/io/druid/benchmark/query/SerializingQueryRunner.java index 256559685524..7b70beb75d0b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SerializingQueryRunner.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SerializingQueryRunner.java @@ -23,7 +23,7 @@ import com.google.common.base.Function; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import java.util.Map; @@ -47,12 +47,12 @@ public SerializingQueryRunner( @Override public Sequence run( - final Query query, + final QueryPlus queryPlus, final Map responseContext ) { return Sequences.map( - baseRunner.run(query, responseContext), + baseRunner.run(queryPlus, responseContext), new Function() { @Override diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java index 07d4daeb8293..6a38eec88e0e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java @@ -47,6 +47,7 @@ import io.druid.segment.TestHelper; import io.druid.segment.column.ValueType; import io.druid.segment.serde.ComplexMetrics; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerConfig; @@ -177,7 +178,8 @@ protected Map getTableMap() Calcites.createRootSchema(druidSchema), walker, CalciteTests.createOperatorTable(), - plannerConfig + plannerConfig, + new ServerConfig() ); groupByQuery = GroupByQuery .builder() diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java index a6a63c8a66c4..5ea9e85bf1ed 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java @@ -92,7 +92,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -155,7 +155,7 @@ private void setupQueries() BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); { // basic.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); @@ -176,7 +176,7 @@ private void setupQueries() basicQueries.put("A", queryA); } { - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); @@ -196,7 +196,7 @@ private void setupQueries() basicQueries.put("timeFilterNumeric", timeFilterQuery); } { - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); @@ -216,7 +216,7 @@ private void setupQueries() basicQueries.put("timeFilterAlphanumeric", timeFilterQuery); } { - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(new Interval(200000, 300000))); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval(200000, 300000))); List queryAggs = new ArrayList<>(); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); queryAggs.add(lsaf); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java index 5e8bab4957fe..7b8db2997dce 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java @@ -89,7 +89,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -156,7 +156,7 @@ private void setupQueries() BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); { // basic.A - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); @@ -176,7 +176,7 @@ private void setupQueries() basicQueries.put("A", queryBuilderA); } { // basic.numericSort - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); @@ -192,7 +192,7 @@ private void setupQueries() basicQueries.put("numericSort", queryBuilderA); } { // basic.alphanumericSort - QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); + QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); diff --git a/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java b/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java index a3733939c903..1305679cd784 100644 --- a/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java +++ b/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java @@ -384,7 +384,7 @@ public void testIntervalBasedTimeGeneration() throws Exception { } - private class RowValueTracker + private static class RowValueTracker { private Map> dimensionMap; diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/IntegerSet.java b/bytebuffer-collections/src/main/java/io/druid/collections/IntegerSet.java index 0509f30fdc64..d2165ff73828 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/IntegerSet.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/IntegerSet.java @@ -19,18 +19,17 @@ package io.druid.collections; -import com.google.common.collect.Sets; import io.druid.collections.bitmap.MutableBitmap; import org.roaringbitmap.IntIterator; +import java.util.AbstractSet; import java.util.Collection; import java.util.Iterator; -import java.util.Set; /** * */ -public class IntegerSet implements Set +public class IntegerSet extends AbstractSet { private final MutableBitmap mutableBitmap; @@ -84,12 +83,6 @@ public Object[] toArray() return retval; } - @Override - public T[] toArray(T[] a) - { - return Sets.newHashSet(this).toArray(a); - } - @Override public boolean add(Integer integer) { diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/bitmap/WrappedConciseBitmap.java b/bytebuffer-collections/src/main/java/io/druid/collections/bitmap/WrappedConciseBitmap.java index 0481e9204884..9e0716f1fc16 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/bitmap/WrappedConciseBitmap.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/bitmap/WrappedConciseBitmap.java @@ -42,7 +42,7 @@ public WrappedConciseBitmap() } /** - * Create a bitmap wrappign the given bitmap + * Create a bitmap wrapping the given bitmap * * @param conciseSet bitmap to be wrapped */ diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/ImmutableNode.java b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/ImmutableNode.java index e29e2002ad7f..203a22c00c40 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/ImmutableNode.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/ImmutableNode.java @@ -160,6 +160,7 @@ public ImmutableBitmap getImmutableBitmap() return bitmapFactory.mapImmutableBitmap(tmpBuffer.asReadOnlyBuffer()); } + @SuppressWarnings("ArgumentParameterSwap") public Iterable getChildren() { return new Iterable() diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/search/RectangularBound.java b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/search/RectangularBound.java index 0d6ab38f7818..1eae8688b05b 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/search/RectangularBound.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/search/RectangularBound.java @@ -78,6 +78,7 @@ public float[] getMaxCoords() return maxCoords; } + @Override @JsonProperty public int getLimit() { diff --git a/ci/travis_script_integration.sh b/ci/travis_script_integration.sh new file mode 100755 index 000000000000..47f62867d394 --- /dev/null +++ b/ci/travis_script_integration.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets 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. + +set -e + +pushd $TRAVIS_BUILD_DIR/integration-tests + +mvn verify -P integration-tests + +popd diff --git a/codestyle/checkstyle.xml b/codestyle/checkstyle.xml index cd8ad3c15de5..860df828adc7 100644 --- a/codestyle/checkstyle.xml +++ b/codestyle/checkstyle.xml @@ -53,5 +53,10 @@ + + + + + diff --git a/common/pom.xml b/common/pom.xml index 24a1a49b457a..d1c6be234351 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -229,4 +229,24 @@ + + + strict + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + -Xep:MissingOverride:WARN + + + + + + + + diff --git a/common/src/main/java/io/druid/annotations/SubclassesMustBePublic.java b/common/src/main/java/io/druid/annotations/SubclassesMustBePublic.java new file mode 100644 index 000000000000..2d8771fe8f09 --- /dev/null +++ b/common/src/main/java/io/druid/annotations/SubclassesMustBePublic.java @@ -0,0 +1,35 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.annotations; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Documented +@Inherited +@Retention(RetentionPolicy.SOURCE) +@Target(ElementType.TYPE) +public @interface SubclassesMustBePublic +{ +} diff --git a/common/src/main/java/io/druid/annotations/SubclassesMustBePublicAnnotationProcessor.java b/common/src/main/java/io/druid/annotations/SubclassesMustBePublicAnnotationProcessor.java new file mode 100644 index 000000000000..2a8f30981072 --- /dev/null +++ b/common/src/main/java/io/druid/annotations/SubclassesMustBePublicAnnotationProcessor.java @@ -0,0 +1,51 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.annotations; + +import javax.annotation.processing.AbstractProcessor; +import javax.annotation.processing.RoundEnvironment; +import javax.annotation.processing.SupportedAnnotationTypes; +import javax.lang.model.element.Element; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.tools.Diagnostic; +import java.util.Set; + +@SupportedAnnotationTypes("io.druid.annotations.SubclassesMustBePublic") +public class SubclassesMustBePublicAnnotationProcessor extends AbstractProcessor +{ + @Override + public boolean process(Set annotations, RoundEnvironment roundEnv) + { + for (TypeElement annotation : annotations) { + Set elementsAnnotatedWith = roundEnv.getElementsAnnotatedWith(annotation); + for (Element element : elementsAnnotatedWith) { + if (!element.getModifiers().contains(Modifier.PUBLIC)) { + processingEnv.getMessager().printMessage( + Diagnostic.Kind.ERROR, + element.getSimpleName() + " must be public", + element + ); + } + } + } + return false; + } +} diff --git a/common/src/main/java/io/druid/collections/CountingMap.java b/common/src/main/java/io/druid/collections/CountingMap.java deleted file mode 100644 index e8bc84531266..000000000000 --- a/common/src/main/java/io/druid/collections/CountingMap.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.collections; - -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableMap; - -import io.druid.java.util.common.guava.DefaultingHashMap; - -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - -/** - */ -public class CountingMap extends DefaultingHashMap -{ - public CountingMap() - { - super(new Supplier() - { - @Override - public AtomicLong get() - { - return new AtomicLong(0); - } - }); - } - - public long add(K key, long value) - { - return get(key).addAndGet(value); - } - - public Map snapshot() - { - final ImmutableMap.Builder builder = ImmutableMap.builder(); - - for (Map.Entry entry : entrySet()) { - builder.put(entry.getKey(), entry.getValue().get()); - } - - return builder.build(); - } -} diff --git a/common/src/main/java/io/druid/common/guava/CombiningSequence.java b/common/src/main/java/io/druid/common/guava/CombiningSequence.java index 6abe7ce02035..6cb465d8960a 100644 --- a/common/src/main/java/io/druid/common/guava/CombiningSequence.java +++ b/common/src/main/java/io/druid/common/guava/CombiningSequence.java @@ -183,16 +183,19 @@ public YieldingAccumulator getAccumulator() return accumulator; } + @Override public void reset() { accumulator.reset(); } + @Override public boolean yielded() { return accumulator.yielded(); } + @Override public void yield() { accumulator.yield(); diff --git a/common/src/main/java/io/druid/common/utils/ServletResourceUtils.java b/common/src/main/java/io/druid/common/utils/ServletResourceUtils.java index dcc37e4d893b..025e1989bbad 100644 --- a/common/src/main/java/io/druid/common/utils/ServletResourceUtils.java +++ b/common/src/main/java/io/druid/common/utils/ServletResourceUtils.java @@ -40,4 +40,12 @@ public static Map sanitizeException(@Nullable Throwable t) t == null ? "null" : (t.getMessage() == null ? t.toString() : t.getMessage()) ); } + + /** + * Converts String errorMsg into a Map so that it produces valid json on serialization into response. + */ + public static Map jsonize(String msgFormat, Object... args) + { + return ImmutableMap.of("error", StringUtils.safeFormat(msgFormat, args)); + } } diff --git a/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java b/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java index c03c52f4a823..9e2205a4fc2d 100644 --- a/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java +++ b/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java @@ -19,13 +19,14 @@ package io.druid.timeline; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; - -import io.druid.java.util.common.guava.Comparators; import io.druid.common.utils.JodaUtils; +import io.druid.java.util.common.guava.Comparators; import io.druid.timeline.partition.ImmutablePartitionHolder; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.PartitionHolder; @@ -37,6 +38,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -89,6 +91,12 @@ public static VersionedIntervalTimeline forSegments(Iterabl return timeline; } + @VisibleForTesting + public Map> getAllTimelineEntries() + { + return allTimelineEntries; + } + public void add(final Interval interval, VersionType version, PartitionChunk object) { try { @@ -159,6 +167,7 @@ public PartitionChunk remove(Interval interval, VersionType version, } } + @Override public PartitionHolder findEntry(Interval interval, VersionType version) { try { @@ -190,6 +199,7 @@ public PartitionHolder findEntry(Interval interval, VersionType vers * @return Holders representing the interval that the objects exist for, PartitionHolders * are guaranteed to be complete */ + @Override public List> lookup(Interval interval) { try { @@ -550,9 +560,9 @@ public class TimelineEntry public TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) { - this.trueInterval = trueInterval; - this.version = version; - this.partitionHolder = partitionHolder; + this.trueInterval = Preconditions.checkNotNull(trueInterval); + this.version = Preconditions.checkNotNull(version); + this.partitionHolder = Preconditions.checkNotNull(partitionHolder); } public Interval getTrueInterval() @@ -569,5 +579,39 @@ public PartitionHolder getPartitionHolder() { return partitionHolder; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final TimelineEntry that = (TimelineEntry) o; + + if (!this.trueInterval.equals(that.trueInterval)) { + return false; + } + + if (!this.version.equals(that.version)) { + return false; + } + + if (!this.partitionHolder.equals(that.partitionHolder)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return Objects.hash(trueInterval, version, partitionHolder); + } } } diff --git a/common/src/main/resources/services/javax.annotation.processing.Processor b/common/src/main/resources/services/javax.annotation.processing.Processor new file mode 100644 index 000000000000..b19a8c2290c3 --- /dev/null +++ b/common/src/main/resources/services/javax.annotation.processing.Processor @@ -0,0 +1 @@ +io.druid.annotations.SubclassesMustBePublicAnnotationProcessor \ No newline at end of file diff --git a/common/src/test/java/io/druid/collections/CountingMapTest.java b/common/src/test/java/io/druid/collections/CountingMapTest.java deleted file mode 100644 index b289727f79a2..000000000000 --- a/common/src/test/java/io/druid/collections/CountingMapTest.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.collections; - -import com.google.common.collect.ImmutableMap; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.concurrent.atomic.AtomicLong; - -public class CountingMapTest -{ - private CountingMap mapObject = null ; - - @Before - public void setUp() - { - mapObject = new CountingMap(); - } - - @After - public void tearDown() - { - mapObject.clear(); - } - - @Test - public void testAdd() - { - long defaultValue = 10; - String defaultKey = "defaultKey"; - long actual; - actual = mapObject.add(defaultKey,defaultValue); - Assert.assertEquals("Values does not match", actual, defaultValue); - } - - @Test - public void testSnapshot() - { - long defaultValue = 10; - String defaultKey = "defaultKey"; - mapObject.add(defaultKey, defaultValue); - ImmutableMap snapShotMap = (ImmutableMap) mapObject.snapshot(); - Assert.assertEquals("Maps size does not match",mapObject.size(),snapShotMap.size()); - long expected = (long) snapShotMap.get(defaultKey); - AtomicLong actual = (AtomicLong) mapObject.get(defaultKey); - Assert.assertEquals("Values for key = " + defaultKey + " does not match", - actual.longValue(),expected); - } -} diff --git a/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java b/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java index 93c8ed71b7f1..3958f53012e7 100644 --- a/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java +++ b/common/src/test/java/io/druid/collections/OrderedMergeSequenceTest.java @@ -37,6 +37,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -367,7 +368,7 @@ public void testMergeMerge() throws Exception ) ); - SequenceTestHelper.testAll(finalMerged, Arrays.asList(1)); + SequenceTestHelper.testAll(finalMerged, Collections.singletonList(1)); } @Test @@ -379,6 +380,6 @@ public void testOne() throws Exception ) ); - SequenceTestHelper.testAll(seq1, Arrays.asList(1)); + SequenceTestHelper.testAll(seq1, Collections.singletonList(1)); } } diff --git a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java index b1f933e0df35..e45c782fd8bb 100644 --- a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java @@ -40,6 +40,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -92,11 +93,11 @@ public void testMerge() throws Exception @Test public void testNoMergeOne() throws Exception { - List> pairs = Arrays.asList( + List> pairs = Collections.singletonList( Pair.of(0, 1) ); - List> expected = Arrays.asList( + List> expected = Collections.singletonList( Pair.of(0, 1) ); @@ -151,7 +152,7 @@ public void testMergeTwo() throws Exception Pair.of(0, 1) ); - List> expected = Arrays.asList( + List> expected = Collections.singletonList( Pair.of(0, 2) ); diff --git a/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java b/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java index 832fdf1d102e..5c4a7646b341 100644 --- a/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java +++ b/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java @@ -56,7 +56,7 @@ public void testUmbrellaIntervalsSimple() throws Exception @Test public void testUmbrellaIntervalsNull() throws Exception { - List intervals = Arrays.asList(); + List intervals = Collections.emptyList(); Throwable thrown = null; try { Interval res = JodaUtils.umbrellaInterval(intervals); diff --git a/common/src/test/java/io/druid/concurrent/ExecsTest.java b/common/src/test/java/io/druid/concurrent/ExecsTest.java index caca1b905f25..fdb75260737c 100644 --- a/common/src/test/java/io/druid/concurrent/ExecsTest.java +++ b/common/src/test/java/io/druid/concurrent/ExecsTest.java @@ -71,6 +71,7 @@ private static void runTest(final int capacity) throws Exception producer.submit( new Runnable() { + @Override public void run() { for (int i = 0; i < nTasks; i++) { diff --git a/common/src/test/java/io/druid/guice/LifecycleScopeTest.java b/common/src/test/java/io/druid/guice/LifecycleScopeTest.java index f08cc8630217..7deefe3a4d39 100644 --- a/common/src/test/java/io/druid/guice/LifecycleScopeTest.java +++ b/common/src/test/java/io/druid/guice/LifecycleScopeTest.java @@ -193,16 +193,19 @@ public void run() ++ran; } + @Override public int getStarted() { return started; } + @Override public int getStopped() { return stopped; } + @Override public int getRan() { return ran; @@ -233,16 +236,19 @@ public void run() ++ran; } + @Override public int getStarted() { return started; } + @Override public int getStopped() { return stopped; } + @Override public int getRan() { return ran; diff --git a/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java b/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java index 8b70b13b0075..6f955fa29330 100644 --- a/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java +++ b/common/src/test/java/io/druid/timeline/VersionedIntervalTimelineTest.java @@ -42,6 +42,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -157,7 +158,7 @@ public void testApril4() throws Exception public void testMay() throws Exception { assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-05-01/2011-05-09", "4", 9) ), timeline.lookup(new Interval("2011-05-01/2011-05-09")) @@ -216,7 +217,7 @@ public void testInsertInWrongOrder() throws Exception add(new Interval(overallStart, overallStart.plus(Days.ONE)), "2", 2); assertValues( - Arrays.asList( + Collections.singletonList( createExpected(oneHourInterval1.toString(), "2", 2) ), timeline.lookup(oneHourInterval1) @@ -380,7 +381,7 @@ public void testInsertAndRemoveSameThingsion() throws Exception { add("2011-05-01/2011-05-10", "5", 10); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-05-01/2011-05-09", "5", 10) ), timeline.lookup(new Interval("2011-05-01/2011-05-09")) @@ -391,7 +392,7 @@ public void testInsertAndRemoveSameThingsion() throws Exception timeline.remove(new Interval("2011-05-01/2011-05-10"), "5", makeSingle(10)) ); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-05-01/2011-05-09", "4", 9) ), timeline.lookup(new Interval("2011-05-01/2011-05-09")) @@ -399,7 +400,7 @@ public void testInsertAndRemoveSameThingsion() throws Exception add("2011-05-01/2011-05-10", "5", 10); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-05-01/2011-05-09", "5", 10) ), timeline.lookup(new Interval("2011-05-01/2011-05-09")) @@ -410,7 +411,7 @@ public void testInsertAndRemoveSameThingsion() throws Exception timeline.remove(new Interval("2011-05-01/2011-05-10"), "4", makeSingle(9)) ); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-05-01/2011-05-09", "5", 10) ), timeline.lookup(new Interval("2011-05-01/2011-05-09")) @@ -450,7 +451,7 @@ public void testOverlapSameVersionIsOkay() throws Exception add("2011-01-01/2011-01-10", "1", 4); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-01-01/2011-01-10", "2", 2) ), timeline.lookup(new Interval("2011-01-01/2011-01-10")) @@ -1177,7 +1178,7 @@ public void testOverlapAndRemove() throws Exception timeline.remove(new Interval("2011-01-10/2011-01-15"), "2", makeSingle(2)); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-01-01/2011-01-20", "1", 1) ), timeline.lookup(new Interval("2011-01-01/2011-01-20")) @@ -1218,7 +1219,7 @@ public void testOverlapAndRemove3() throws Exception timeline.remove(new Interval("2011-01-10/2011-01-14"), "2", makeSingle(3)); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-01-01/2011-01-20", "1", 1) ), @@ -1258,7 +1259,7 @@ public void testOverlapAndRemove5() throws Exception add("2011-01-01/2011-01-20", "1", 1); assertValues( - Arrays.asList( + Collections.singletonList( createExpected("2011-01-01/2011-01-20", "1", 1) ), timeline.lookup(new Interval("2011-01-01/2011-01-20")) diff --git a/distribution/pom.xml b/distribution/pom.xml index 6be0e80ad445..0636378d0343 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -97,10 +97,12 @@ -c io.druid.extensions:druid-lookups-cached-single -c + io.druid.extensions:druid-protobuf-extensions + -c io.druid.extensions:mysql-metadata-storage -c io.druid.extensions:postgresql-metadata-storage - -c + -c io.druid.extensions:druid-kerberos -c io.druid.extensions:druid-s3-extensions @@ -201,7 +203,6 @@ -l ${settings.localRepository} --no-default-hadoop - ${druid.distribution.pulldeps.opts} -c io.druid.extensions.contrib:ambari-metrics-emitter -c diff --git a/distribution/src/assembly/assembly.xml b/distribution/src/assembly/assembly.xml index 9e7853a8aefd..02a71d3ede65 100644 --- a/distribution/src/assembly/assembly.xml +++ b/distribution/src/assembly/assembly.xml @@ -114,6 +114,13 @@ conf-quickstart/tranquility + + ../examples/quickstart/protobuf + + * + + quickstart/protobuf + ../examples/conf diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index fa6a01b56476..98fa02f129cd 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -34,9 +34,10 @@ Druid uses Jetty to serve HTTP requests. |Property|Description|Default| |--------|-----------|-------| -|`druid.server.http.numThreads`|Number of threads for HTTP requests.|10| +|`druid.server.http.numThreads`|Number of threads for HTTP requests.|max(10, (Number of cores * 17) / 16 + 2) + 30| |`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| |`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000| +|`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data nodes such as historicals and realtime processes to execute a query. This is an advance configuration that allows to protect in case broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is not necessarily bad if broker is never under heavy concurrent load in which case data gathered is processed quickly and freeing up the memory used.|Long.MAX_VALUE| |`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time processes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|20| |`druid.broker.http.compressionCodec`|Compression codec the Broker uses to communicate with historical and real-time processes. May be "gzip" or "identity".|gzip| |`druid.broker.http.readTimeout`|The timeout for data reads from historical and real-time processes.|PT15M| @@ -86,6 +87,7 @@ See [groupBy server configuration](../querying/groupbyquery.html#server-configur |Property|Description|Default| |--------|-----------|-------| |`druid.query.segmentMetadata.defaultHistory`|When no interval is specified in the query, use a default interval of defaultHistory before the end time of the most recent segment, specified in ISO8601 format. This property also controls the duration of the default interval used by GET /druid/v2/datasources/{dataSourceName} interactions for retrieving datasource dimensions/metrics.|P1W| +|`druid.query.segmentMetadata.defaultAnalysisTypes`|This can be used to set the Default Analysis Types for all segment metadata queries, this can be overridden when making the query|["cardinality", "interval", "minmax"]| ### SQL diff --git a/docs/content/configuration/historical.md b/docs/content/configuration/historical.md index e8b8832a4786..26f268af7779 100644 --- a/docs/content/configuration/historical.md +++ b/docs/content/configuration/historical.md @@ -46,7 +46,7 @@ Druid uses Jetty to serve HTTP requests. |Property|Description|Default| |--------|-----------|-------| -|`druid.server.http.numThreads`|Number of threads for HTTP requests.|10| +|`druid.server.http.numThreads`|Number of threads for HTTP requests.|max(10, (Number of cores * 17) / 16 + 2) + 30| |`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| |`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000| diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index af1c5eb035ad..4a259b1d5500 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -23,9 +23,10 @@ Many of Druid's external dependencies can be plugged in as modules. Extensions c |--------|-----------|-------| |`druid.extensions.directory`|The root extension directory where user can put extensions related files. Druid will load extensions stored under this directory.|`extensions` (This is a relative path to Druid's working directory)| |`druid.extensions.hadoopDependenciesDir`|The root hadoop dependencies directory where user can put hadoop related dependencies files. Druid will load the dependencies based on the hadoop coordinate specified in the hadoop index task.|`hadoop-dependencies` (This is a relative path to Druid's working directory| -|`druid.extensions.hadoopContainerDruidClasspath`|Hadoop Indexing launches hadoop jobs and this configuration provides way to explicitly set the user classpath for the hadoop job. By default this is computed automatically by druid based on the druid process classpath and set of extensions. However, sometimes you might want to be explicit to resolve dependency conflicts between druid and hadoop.|null| |`druid.extensions.loadList`|A JSON array of extensions to load from extension directories by Druid. If it is not specified, its value will be `null` and Druid will load all the extensions under `druid.extensions.directory`. If its value is empty list `[]`, then no extensions will be loaded at all. It is also allowed to specify absolute path of other custom extensions not stored in the common extensions directory.|null| |`druid.extensions.searchCurrentClassloader`|This is a boolean flag that determines if Druid will search the main classloader for extensions. It defaults to true but can be turned off if you have reason to not automatically add all modules on the classpath.|true| +|`druid.extensions.hadoopContainerDruidClasspath`|Hadoop Indexing launches hadoop jobs and this configuration provides way to explicitly set the user classpath for the hadoop job. By default this is computed automatically by druid based on the druid process classpath and set of extensions. However, sometimes you might want to be explicit to resolve dependency conflicts between druid and hadoop.|null| +|`druid.extensions.addExtensionsToHadoopContainer`|Only applicable if `druid.extensions.hadoopContainerDruidClasspath` is provided. If set to true, then extensions specified in the loadList are added to hadoop container classpath. Note that when `druid.extensions.hadoopContainerDruidClasspath` is not provided then extensions are always added to hadoop container classpath.|false| ### Zookeeper We recommend just setting the base ZK path and the ZK service host, but all ZK paths that Druid uses can be overwritten to absolute paths. diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md index 39fb060ecd50..5c4aa430c606 100644 --- a/docs/content/design/coordinator.md +++ b/docs/content/design/coordinator.md @@ -55,6 +55,13 @@ Returns the Druid version, loaded extensions, memory used, total memory and othe Returns the current leader coordinator of the cluster. +* `/druid/coordinator/v1/isLeader` + +Returns a JSON object with field "leader", either true or false, indicating if this server is the current leader +coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not. +This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service +at the load balancer. + * `/druid/coordinator/v1/loadstatus` Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster. @@ -153,7 +160,7 @@ Returns a map of an interval to a map of segment metadata to a set of server nam * `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` -Returns a set of segment ids for an ISO8601 interval. Note that the interval is delimited by a `_` instead of a `/` +Returns a set of segment ids for an ISO8601 interval. Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28). * `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?simple` @@ -216,6 +223,8 @@ Returns all rules for a specified datasource and includes default datasource. #### Intervals +Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28). + * `/druid/coordinator/v1/intervals` Returns all intervals for all datasources with total size and count. @@ -271,6 +280,8 @@ Disables a datasource. Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource. +Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2016-06-27_2016-06-28). + * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` Disables a segment. diff --git a/docs/content/design/indexing-service.md b/docs/content/design/indexing-service.md index 58cbbbe6d8e9..43087b7e379c 100644 --- a/docs/content/design/indexing-service.md +++ b/docs/content/design/indexing-service.md @@ -33,6 +33,26 @@ In local mode overlord is also responsible for creating peons for executing task Local mode is typically used for simple workflows. In remote mode, the overlord and middle manager are run in separate processes and you can run each on a different server. This mode is recommended if you intend to use the indexing service as the single endpoint for all Druid indexing. +#### Leadership status + +If you have multiple overlords, just one is leading at any given time. The others are on standby. To get the current +leader overlord of the cluster, call: + + +``` +http://:/druid/indexer/v1/leader +``` + +To see if a given server is the current leader overlord of the cluster, call: + +``` +http://:/druid/indexer/v1/isLeader +``` + +This returns a JSON object with field "leader", either true or false. In addition, this call returns HTTP 200 if the +server is the current leader and HTTP 404 if not. This is suitable for use as a load balancer status check if you +only want the active leader to be considered in-service at the load balancer. + #### Submitting Tasks and Querying Task Status Tasks are submitted to the overlord node in the form of JSON objects. Tasks can be submitted via POST requests to: diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md index 497735c7ddf9..3a3946accf48 100644 --- a/docs/content/development/extensions-contrib/azure.md +++ b/docs/content/development/extensions-contrib/azure.md @@ -51,10 +51,18 @@ Sample spec: } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. + |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-azure-blobstore".|N/A|yes| +|type|This should be `static-azure-blobstore`.|N/A|yes| |blobs|JSON array of [Azure blobs](https://msdn.microsoft.com/en-us/library/azure/ee691964.aspx).|N/A|yes| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Azure objects.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching an Azure object.|60000|no| +|maxFetchRetry|Maximum retry for fetching an Azure object.|3|no| Azure Blobs: diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md index de56c6cc79cc..954179d5fbdb 100644 --- a/docs/content/development/extensions-contrib/cloudfiles.md +++ b/docs/content/development/extensions-contrib/cloudfiles.md @@ -51,11 +51,18 @@ Sample spec: ] } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-cloudfiles".|N/A|yes| +|type|This should be `static-cloudfiles`.|N/A|yes| |blobs|JSON array of Cloud Files blobs.|N/A|yes| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache.|1073741824|no| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| +|fetchTimeout|Timeout for fetching a Cloud Files object.|60000|no| +|maxFetchRetry|Maximum retry for fetching a Cloud Files object.|3|no| Cloud Files Blobs: diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md index 1c080c5c3dd4..7aa764728828 100644 --- a/docs/content/development/extensions-contrib/google.md +++ b/docs/content/development/extensions-contrib/google.md @@ -41,10 +41,18 @@ Sample spec: } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. + |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-google-blobstore".|N/A|yes| +|type|This should be `static-google-blobstore`.|N/A|yes| |blobs|JSON array of Google Blobs.|N/A|yes| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching Google Blobs.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching a Google Blob.|60000|no| +|maxFetchRetry|Maximum retry for fetching a Google Blob.|3|no| Google Blobs: diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 5cb3232cf887..0ce192c11b8e 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -122,7 +122,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' below for more details.|no| |`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| |`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| -|`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)| +|`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. This option is deprecated. Use `completionTimeout` of KafkaSupervisorIOConfig instead.|no (default == 0)| |`resetOffsetAutomatically`|Boolean|Whether to reset the consumer offset if the next offset that it is trying to fetch is less than the earliest available offset for that particular partition. The consumer offset will be reset to either the earliest or latest offset depending on `useEarliestOffset` property of `KafkaSupervisorIOConfig` (see below). This situation typically occurs when messages in Kafka are no longer available for consumption and therefore won't be ingested into Druid. If set to false then ingestion for that particular partition will halt and manual intervention is required to correct the situation, please see `Reset Supervisor` API below.|no (default == false)| |`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| |`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| @@ -168,6 +168,7 @@ For Roaring bitmaps: |`useEarliestOffset`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| |`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT30M)| |`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)| +|`skipOffsetGaps`|Boolean|Whether or not to allow gaps of missing offsets in the Kafka stream. This is required for compatibility with implementations such as MapR Streams which does not guarantee consecutive offsets. If this is false, an exception will be thrown if offsets are not consecutive.|no (default == false)| ## Supervisor API diff --git a/docs/content/development/extensions-core/lookups-cached-global.md b/docs/content/development/extensions-core/lookups-cached-global.md index 72d346201c1d..385cc07ad7cd 100644 --- a/docs/content/development/extensions-core/lookups-cached-global.md +++ b/docs/content/development/extensions-core/lookups-cached-global.md @@ -195,12 +195,17 @@ The `namespaceParseSpec` can be one of a number of values. Each of the examples Only ONE file which matches the search will be used. For most implementations, the discriminator for choosing the URIs is by whichever one reports the most recent timestamp for its modification time. ### csv lookupParseSpec - |Parameter|Description|Required|Default| |---------|-----------|--------|-------| -|`columns`|The list of columns in the csv file|yes|`null`| +|`columns`|The list of columns in the csv file|no if `hasHeaderRow` is set|`null`| |`keyColumn`|The name of the column containing the key|no|The first column| |`valueColumn`|The name of the column containing the value|no|The second column| +|`hasHeaderRow`|A flag to indicate that column information can be extracted from the input files' header row|no|false| +|`skipHeaderRows`|Number of header rows to be skipped|no|0| + +If both `skipHeaderRows` and `hasHeaderRow` options are set, `skipHeaderRows` is first applied. For example, if you set +`skipHeaderRows` to 2 and `hasHeaderRow` to true, Druid will skip the first two lines and then extract column information +from the third line. *example input* @@ -222,15 +227,19 @@ truck,something3,buck ``` ### tsv lookupParseSpec - |Parameter|Description|Required|Default| |---------|-----------|--------|-------| -|`columns`|The list of columns in the csv file|yes|`null`| +|`columns`|The list of columns in the tsv file|yes|`null`| |`keyColumn`|The name of the column containing the key|no|The first column| |`valueColumn`|The name of the column containing the value|no|The second column| |`delimiter`|The delimiter in the file|no|tab (`\t`)| |`listDelimiter`|The list delimiter in the file|no| (`\u0001`)| +|`hasHeaderRow`|A flag to indicate that column information can be extracted from the input files' header row|no|false| +|`skipHeaderRows`|Number of header rows to be skipped|no|0| +If both `skipHeaderRows` and `hasHeaderRow` options are set, `skipHeaderRows` is first applied. For example, if you set +`skipHeaderRows` to 2 and `hasHeaderRow` to true, Druid will skip the first two lines and then extract column information +from the third line. *example input* diff --git a/docs/content/development/extensions-core/protobuf.md b/docs/content/development/extensions-core/protobuf.md new file mode 100644 index 000000000000..c45e47c6ecd5 --- /dev/null +++ b/docs/content/development/extensions-core/protobuf.md @@ -0,0 +1,203 @@ +--- +layout: doc_page +--- + +# Protobuf + +This extension enables Druid to ingest and understand the Protobuf data format. Make sure to [include](../../operations/including-extensions.html) `druid-protobuf-extensions` as an extension. + +## Protobuf Parser + + +| Field | Type | Description | Required | +|-------|------|-------------|----------| +| type | String | This should say `protobuf`. | no | +| descriptor | String | Protobuf descriptor file name in the classpath or URL. | yes | +| protoMessageType | String | Protobuf message type in the descriptor. Both short name and fully qualified name are accepted. The parser uses the first message type found in the descriptor if not specified. | no | +| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. The format must be json. See [JSON ParseSpec](../../ingestion/index.html) for more configuration options. Please note timeAndDims parseSpec is no longer supported. | yes | + +## Example: Load Protobuf messages from Kafka + +This example demonstrates how to load Protobuf messages from Kafka. Please read the [Load from Kafka tutorial](../../tutorial/tutorial-kafka.html) first. This example will use the same "metrics" dataset. + +Files used in this example are found at `./examples/quickstart/protobuf` in your Druid directory. + +- We will use [Kafka Indexing Service](./kafka-ingestion.html) instead of Tranquility. +- Kafka broker host is `localhost:9092`. +- Kafka topic is `metrics_pb` instead of `metrics`. +- datasource name is `metrics-kafka-pb` instead of `metrics-kafka` to avoid the confusion. + +Here is the metrics JSON example. + +```json +{ + "unit": "milliseconds", + "http_method": "GET", + "value": 44, + "timestamp": "2017-04-06T02:36:22Z", + "http_code": "200", + "page": "/", + "metricType": "request/latency", + "server": "www1.example.com" +} +``` + +### Proto file + +The proto file should look like this. Save it as metrics.proto. + +``` +syntax = "proto3"; +message Metrics { + string unit = 1; + string http_method = 2; + int32 value = 3; + string timestamp = 4; + string http_code = 5; + string page = 6; + string metricType = 7; + string server = 8; +} +``` + +### Descriptor file + +Using the `protoc` Protobuf compiler to generate the descriptor file. Save the metrics.desc file either in the classpath or reachable by URL. In this example the descriptor file was saved at /tmp/metrics.desc. + +``` +protoc -o /tmp/metrics.desc metrics.proto +``` + +### Supervisor spec JSON + +Below is the complete Supervisor spec JSON to be submitted to the Overlord. +Please make sure these keys are properly configured for successful ingestion. + +- `descriptor` for the descriptor file URL. +- `protoMessageType` from the proto definition. +- parseSpec `format` must be `json`. +- `topic` to subscribe. The topic is "metrics_pb" instead of "metrics". +- `bootstrap.server` is the kafka broker host. + +```json +{ + "type": "kafka", + "dataSchema": { + "dataSource": "metrics-kafka2", + "parser": { + "type": "protobuf", + "descriptor": "file:///tmp/metrics.desc", + "protoMessageType": "Metrics", + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "unit", + "http_method", + "http_code", + "page", + "metricType", + "server" + ], + "dimensionExclusions": [ + "timestamp", + "value" + ] + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kafka", + "maxRowsPerSegment": 5000000 + }, + "ioConfig": { + "topic": "metrics_pb", + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H" + } +} +``` + +## Kafka Producer + +Here is the sample script that publishes the metrics to Kafka in Protobuf format. + +1. Run `protoc` again with the Python binding option. This command generates `metrics_pb2.py` file. + ``` + protoc -o metrics.desc metrics.proto --python_out=. + ``` + +2. Create Kafka producer script. + +This script requires `protobuf` and `kafka-python` modules. + +```python +#!/usr/bin/env python + +import sys +import json + +from kafka import KafkaProducer +from metrics_pb2 import Metrics + +producer = KafkaProducer(bootstrap_servers='localhost:9092') +topic = 'metrics_pb' +metrics = Metrics() + +for row in iter(sys.stdin): + d = json.loads(row) + for k, v in d.items(): + setattr(metrics, k, v) + pb = metrics.SerializeToString() + producer.send(topic, pb) +``` + +3. run producer + +``` +./bin/generate-example-metrics | ./pb_publisher.py +``` + +4. test + +``` +kafka-console-consumer --zookeeper localhost --topic metrics_pb +``` + +It should print messages like this +> millisecondsGETR"2017-04-06T03:23:56Z*2002/list:request/latencyBwww1.example.com diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index 59f6a7084b3b..f24a406946b6 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -32,7 +32,16 @@ Sample spec: } ``` +This firehose provides caching and prefetching features. In IndexTask, a firehose can be read twice if intervals or +shardSpecs are not specified, and, in this case, caching can be useful. Prefetching is preferred when direct scan of objects is slow. + |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should be "static-s3"|N/A|yes| -|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|yes| +|type|This should be `static-s3`.|N/A|yes| +|uris|JSON array of URIs where s3 files to be ingested are located.|N/A|`uris` or `prefixes` must be set| +|prefixes|JSON array of URI prefixes for the locations of s3 files to be ingested.|N/A|`uris` or `prefixes` must be set| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824|no| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824|no| +|prefetchTriggerBytes|Threshold to trigger prefetching s3 objects.|maxFetchCapacityBytes / 2|no| +|fetchTimeout|Timeout for fetching an s3 object.|60000|no| +|maxFetchRetry|Maximum retry for fetching an s3 object.|3|no| \ No newline at end of file diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 779ea6b65d99..d6bfca71dba0 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -32,6 +32,7 @@ Core extensions are maintained by Druid committers. |druid-kerberos|Kerberos authentication for druid nodes.|[link](../development/extensions-core/druid-kerberos.html)| |druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.html)| |druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.html)| +|druid-protobuf-extensions| Support for data in Protobuf data format.|[link](../development/extensions-core/protobuf.html)| |druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)| |druid-stats|Statistics related module including variance and standard deviation.|[link](../development/extensions-core/stats.html)| |mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)| diff --git a/docs/content/ingestion/data-formats.md b/docs/content/ingestion/data-formats.md index 95807cc60a6a..859f75e12be8 100644 --- a/docs/content/ingestion/data-formats.md +++ b/docs/content/ingestion/data-formats.md @@ -72,10 +72,8 @@ If you have nested JSON, [Druid can automatically flatten it for you](flatten-js ### CSV -Since the CSV data cannot contain the column names (no header is allowed), these must be added before that data can be processed: - ```json - "parseSpec":{ + "parseSpec": { "format" : "csv", "timestampSpec" : { "column" : "timestamp" @@ -87,12 +85,27 @@ Since the CSV data cannot contain the column names (no header is allowed), these } ``` -The `columns` field must match the columns of your input data in the same order. +#### CSV Index Tasks + +If your input files contain a header, the `columns` field is optional and you don't need to set. +Instead, you can set the `hasHeaderRow` field to true, which makes Druid automatically extract the column information from the header. +Otherwise, you must set the `columns` field and ensure that field must match the columns of your input data in the same order. + +Also, you can skip some header rows by setting `skipHeaderRows` in your parseSpec. If both `skipHeaderRows` and `hasHeaderRow` options are set, +`skipHeaderRows` is first applied. For example, if you set `skipHeaderRows` to 2 and `hasHeaderRow` to true, Druid will +skip the first two lines and then extract column information from the third line. -### TSV +Note that `hasHeaderRow` and `skipHeaderRows` are effective only for non-Hadoop batch index tasks. Other types of index +tasks will fail with an exception. + +#### Other CSV Ingestion Tasks + +The `columns` field must be included and and ensure that the order of the fields matches the columns of your input data in the same order. + +### TSV (Delimited) ```json - "parseSpec":{ + "parseSpec": { "format" : "tsv", "timestampSpec" : { "column" : "timestamp" @@ -105,10 +118,25 @@ The `columns` field must match the columns of your input data in the same order. } ``` -The `columns` field must match the columns of your input data in the same order. - Be sure to change the `delimiter` to the appropriate delimiter for your data. Like CSV, you must specify the columns and which subset of the columns you want indexed. +#### TSV (Delimited) Index Tasks + +If your input files contain a header, the `columns` field is optional and you don't need to set. +Instead, you can set the `hasHeaderRow` field to true, which makes Druid automatically extract the column information from the header. +Otherwise, you must set the `columns` field and ensure that field must match the columns of your input data in the same order. + +Also, you can skip some header rows by setting `skipHeaderRows` in your parseSpec. If both `skipHeaderRows` and `hasHeaderRow` options are set, +`skipHeaderRows` is first applied. For example, if you set `skipHeaderRows` to 2 and `hasHeaderRow` to true, Druid will +skip the first two lines and then extract column information from the third line. + +Note that `hasHeaderRow` and `skipHeaderRows` are effective only for non-Hadoop batch index tasks. Other types of index +tasks will fail with an exception. + +#### Other TSV (Delimited) Ingestion Tasks + +The `columns` field must be included and and ensure that the order of the fields matches the columns of your input data in the same order. + ### Regex ```json diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md index 91d594ec8c10..aec3ea15f62d 100644 --- a/docs/content/ingestion/firehose.md +++ b/docs/content/ingestion/firehose.md @@ -36,6 +36,28 @@ A sample local firehose spec is shown below: |filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter.html) for more information.|yes| |baseDir|directory to search recursively for files to be ingested. |yes| +#### HttpFirehose + +This Firehose can be used to read the data from remote sites via HTTP. +A sample http firehose spec is shown below: + +```json +{ + "type" : "http", + "uris" : ["http://example.com/uri1", "http://example2.com/uri2"] +} +``` + +The below configurations can be optionally used for tuning the firehose performance. + +|property|description|default| +|--------|-----------|-------| +|maxCacheCapacityBytes|Maximum size of the cache space in bytes. 0 means disabling cache. Cached files are not removed until the ingestion task completes.|1073741824| +|maxFetchCapacityBytes|Maximum size of the fetch space in bytes. 0 means disabling prefetch. Prefetched files are removed immediately once they are read.|1073741824| +|prefetchTriggerBytes|Threshold to trigger prefetching http objects.|maxFetchCapacityBytes / 2| +|fetchTimeout|Timeout for fetching a http object.|60000| +|maxFetchRetry|Maximum retry for fetching a http object.|3| + #### IngestSegmentFirehose This Firehose can be used to read the data from existing druid segments. diff --git a/docs/content/ingestion/index.md b/docs/content/ingestion/index.md index 6caf9b133672..58874055166b 100644 --- a/docs/content/ingestion/index.md +++ b/docs/content/ingestion/index.md @@ -109,13 +109,6 @@ If `type` is not included, the parser defaults to `string`. For additional data | type | String | This should say `string` in general, or `hadoopyString` when used in a Hadoop indexing job. | no | | parseSpec | JSON Object | Specifies the format, timestamp, and dimensions of the data. | yes | -### Protobuf Parser - -| Field | Type | Description | Required | -|-------|------|-------------|----------| -| type | String | This should say `protobuf`. | no | -| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a timeAndDims parseSpec. | yes | - ### ParseSpec ParseSpecs serve two purposes: diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 6e01e82a386c..835c374ecc9a 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -104,7 +104,6 @@ See [Ingestion](../ingestion/index.html) |type|The task type, this should always be "index".|none|yes| |firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes| |appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no| -|skipFirehoseCaching|By default the IndexTask will fully read the supplied firehose to disk before processing the data. This prevents the task from doing multiple remote fetches and enforces determinism if more than one pass through the data is required. It also allows the task to retry fetching the data if the firehose throws an exception during reading. This requires sufficient disk space for the temporary cache.|false|no| #### TuningConfig @@ -115,12 +114,15 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |type|The task type, this should always be "index".|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|75000|no| +|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| |buildV9Directly|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|true|no| |forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| +|forceGuaranteedRollup|Forces guaranteeing the perfect rollup. The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with `appendToExisting` of IOConfig and `forceExtendableShardSpecs`.|false|no| |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| +|publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| #### IndexSpec diff --git a/docs/content/operations/metrics.md b/docs/content/operations/metrics.md index 0bf7abca6317..ea5679510837 100644 --- a/docs/content/operations/metrics.md +++ b/docs/content/operations/metrics.md @@ -120,6 +120,8 @@ These metrics are only available if the RealtimeMetricsMonitor is included in th |`ingest/handoff/count`|Number of handoffs that happened.|dataSource.|Varies. Generally greater than 0 once every segment granular period if cluster operating normally| |`ingest/sink/count`|Number of sinks not handoffed.|dataSource.|1~3| |`ingest/events/messageGap`|Time gap between the data time in event and current system time.|dataSource.|Greater than 0, depends on the time carried in event | +|`ingest/kafka/lag`|Applicable for Kafka Indexing Service. Total lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute.|dataSource.|Greater than 0, should not be a very high number | + Note: If the JVM does not support CPU time measurement for the current thread, ingest/merge/cpu and ingest/persists/cpu will be 0. diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md index 19ab9cd64287..236e13b5a7a0 100644 --- a/docs/content/querying/dimensionspecs.md +++ b/docs/content/querying/dimensionspecs.md @@ -529,20 +529,3 @@ The following extraction function creates buckets of 5 starting from 2. In this "offset" : 2 } ``` - -### Bucket Extraction Function - -Bucket extraction function is used to bucket numerical values in each range of the given size by converting them to the same base value. Non numeric values are converted to null. - -* `size` : the size of the buckets (optional, default 1) -* `offset` : the offset for the buckets (optional, default 0) - -The following extraction function creates buckets of 5 starting from 2. In this case, values in the range of [2, 7) will be converted to 2, values in [7, 12) will be converted to 7, etc. - -```json -{ - "type" : "bucket", - "size" : 5, - "offset" : 2 -} -``` diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 82f2d81a39e1..32bbbf42492c 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -245,6 +245,7 @@ When using the "v2" strategy, the following query context parameters apply: |`maxMergingDictionarySize`|Can be used to lower the value of `druid.query.groupBy.maxMergingDictionarySize` for this query.| |`maxOnDiskStorage`|Can be used to lower the value of `druid.query.groupBy.maxOnDiskStorage` for this query.| |`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.| +|`forcePushDownLimit`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.| When using the "v1" strategy, the following query context parameters apply: diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 080d71f19c7d..23ef47f6549e 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -67,6 +67,7 @@ The configuration is propagated to the query serving nodes (broker / router / pe The query serving nodes have an internal API for managing lookups on the node and those are used by the coordinator. The coordinator periodically checks if any of the nodes need to load/drop lookups and updates them appropriately. +# API for configuring lookups ## Bulk update Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups`. The format of the json object is as follows: @@ -232,6 +233,27 @@ To discover a list of tiers currently active in the cluster **instead of** ones ## List lookup names A `GET` to `/druid/coordinator/v1/lookups/{tier}` will return a list of known lookup names for that tier. +# Additional API related to status of configured lookups +These end points can be used to get the propagation status of configured lookups to lookup nodes such as historicals. + +## List load status of all lookups +`GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`. + +## List load status of lookups in a tier +`GET /druid/coordinator/v1/lookups/status/{tier}` with optional query parameter `detailed`. + +## List load status of single lookup +`GET /druid/coordinator/v1/lookups/status/{tier}/{lookup}` with optional query parameter `detailed`. + +## List lookup state of all nodes +`GET /druid/coordinator/v1/lookups/nodeStatus` with optional query parameter `discover` to discover tiers from zookeeper or configured lookup tiers are listed. + +## List lookup state of nodes in a tier +`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}` + +## List lookup state of single node +`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}/{host:port}` + # Internal API The Peon, Router, Broker, and Historical nodes all have the ability to consume lookup configuration. diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index d53a239ad8cc..440c6d8c87bc 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -10,6 +10,7 @@ The query context is used for various query configuration parameters. The follow |property |default | description | |-----------------|----------------------------------------|----------------------| |timeout | `druid.server.http.defaultQueryTimeout`| Query timeout in millis, beyond which unfinished queries will be cancelled. 0 timeout means `no timeout`. To set the default timeout, see [broker configuration](broker.html) | +|maxScatterGatherBytes| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data nodes such as historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [broker configuration](broker.html) for more details.| |priority | `0` | Query Priority. Queries with higher priority get precedence for computational resources.| |queryId | auto-generated | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query | |useCache | `true` | Flag indicating whether to leverage the query cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useCache or druid.historical.cache.useCache to determine whether or not to read from the query cache | diff --git a/docs/content/querying/segmentmetadataquery.md b/docs/content/querying/segmentmetadataquery.md index 820db6cadfcf..e04edfc17b22 100644 --- a/docs/content/querying/segmentmetadataquery.md +++ b/docs/content/querying/segmentmetadataquery.md @@ -32,7 +32,7 @@ There are several main parts to a segment metadata query: |toInclude|A JSON Object representing what columns should be included in the result. Defaults to "all".|no| |merge|Merge all individual segment metadata results into a single result|no| |context|See [Context](../querying/query-context.html)|no| -|analysisTypes|A list of Strings specifying what column properties (e.g. cardinality, size) should be calculated and returned in the result. Defaults to ["cardinality", "interval", "minmax"]. See section [analysisTypes](#analysistypes) for more details.|no| +|analysisTypes|A list of Strings specifying what column properties (e.g. cardinality, size) should be calculated and returned in the result. Defaults to ["cardinality", "interval", "minmax"], but can be overridden with using this [BrokerConfig](../configuration/broker.html#segment-metadata-query-config). See section [analysisTypes](#analysistypes) for more details.|no| |lenientAggregatorMerge|If true, and if the "aggregators" analysisType is enabled, aggregators will be merged leniently. See below for details.|no| The format of the result is: @@ -106,7 +106,10 @@ The grammar is as follows: This is a list of properties that determines the amount of information returned about the columns, i.e. analyses to be performed on the columns. -By default, the "cardinality", "size", "interval", and "minmax" types will be used. If a property is not needed, omitting it from this list will result in a more efficient query. +By default, the "cardinality", "interval", and "minmax" types will be used. If a property is not needed, omitting it from this list will result in a more efficient query. + +The default analysis types can be set in the broker configuration via: + `druid.query.segmentMetadata.defaultAnalysisTypes` Types of column analyses are described below: diff --git a/docs/content/querying/select-query.md b/docs/content/querying/select-query.md index 1a6cce58dce1..41960d30a97e 100644 --- a/docs/content/querying/select-query.md +++ b/docs/content/querying/select-query.md @@ -30,6 +30,7 @@ There are several main parts to a select query: |filter|See [Filters](../querying/filters.html)|no| |dimensions|A JSON list of dimensions to select; or see [DimensionSpec](../querying/dimensionspecs.html) for ways to extract dimensions. If left empty, all dimensions are returned.|no| |metrics|A String array of metrics to select. If left empty, all metrics are returned.|no| +|granularity|Defines the granularity of the query. See [Granularities](../querying/granularities.html)|yes| |pagingSpec|A JSON object indicating offsets into different scanned segments. Query results will return a `pagingIdentifiers` value that can be reused in the next query for pagination.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| diff --git a/examples/conf-quickstart/druid/middleManager/runtime.properties b/examples/conf-quickstart/druid/middleManager/runtime.properties index 422b94387ffc..dc1f6aceddb3 100644 --- a/examples/conf-quickstart/druid/middleManager/runtime.properties +++ b/examples/conf-quickstart/druid/middleManager/runtime.properties @@ -17,4 +17,4 @@ druid.processing.numThreads=2 # Hadoop indexing druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp -druid.indexer.task.defaultHadoopCoordinates=["org.apache.hadoop:hadoop-client:2.3.0"] +druid.indexer.task.defaultHadoopCoordinates=["org.apache.hadoop:hadoop-client:2.7.3"] diff --git a/examples/conf/druid/middleManager/runtime.properties b/examples/conf/druid/middleManager/runtime.properties index 84945838d6ef..ce857a0d09d8 100644 --- a/examples/conf/druid/middleManager/runtime.properties +++ b/examples/conf/druid/middleManager/runtime.properties @@ -17,4 +17,4 @@ druid.processing.numThreads=2 # Hadoop indexing druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp -druid.indexer.task.defaultHadoopCoordinates=["org.apache.hadoop:hadoop-client:2.3.0"] +druid.indexer.task.defaultHadoopCoordinates=["org.apache.hadoop:hadoop-client:2.7.3"] diff --git a/examples/quickstart/protobuf/kafka-metrics-pb.json b/examples/quickstart/protobuf/kafka-metrics-pb.json new file mode 100644 index 000000000000..0b896a8b9370 --- /dev/null +++ b/examples/quickstart/protobuf/kafka-metrics-pb.json @@ -0,0 +1,71 @@ +{ + "type": "kafka", + "dataSchema": { + "dataSource": "metrics-kafka2", + "parser": { + "type": "protobuf", + "descriptor": "file:///tmp/metrics.desc", + "protoMessageType": "Metrics", + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "unit", + "http_method", + "http_code", + "page", + "metricType", + "server" + ], + "dimensionExclusions": [ + "timestamp", + "value" + ] + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kafka", + "maxRowsPerSegment": 5000000 + }, + "ioConfig": { + "topic": "metrics_pb", + "consumerProperties": { + "bootstrap.servers": "localhost:9092" + }, + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H" + } +} diff --git a/examples/quickstart/protobuf/metrics.desc b/examples/quickstart/protobuf/metrics.desc new file mode 100644 index 000000000000..4def5e6fe8d1 --- /dev/null +++ b/examples/quickstart/protobuf/metrics.desc @@ -0,0 +1,15 @@ + +õ + metrics.proto"Û +Metrics +unit ( Runit + http_method ( R +httpMethod +value (Rvalue + timestamp ( R timestamp + http_code ( RhttpCode +page ( Rpage + +metricType ( R +metricType +server ( Rserverbproto3 \ No newline at end of file diff --git a/examples/quickstart/protobuf/metrics.proto b/examples/quickstart/protobuf/metrics.proto new file mode 100644 index 000000000000..76f9d1ce2611 --- /dev/null +++ b/examples/quickstart/protobuf/metrics.proto @@ -0,0 +1,11 @@ +syntax = "proto3"; +message Metrics { + string unit = 1; + string http_method = 2; + int32 value = 3; + string timestamp = 4; + string http_code = 5; + string page = 6; + string metricType = 7; + string server = 8; +} diff --git a/examples/quickstart/protobuf/metrics_pb2.py b/examples/quickstart/protobuf/metrics_pb2.py new file mode 100644 index 000000000000..af4c5d24ff2e --- /dev/null +++ b/examples/quickstart/protobuf/metrics_pb2.py @@ -0,0 +1,118 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: metrics.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +from google.protobuf import descriptor_pb2 +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='metrics.proto', + package='', + syntax='proto3', + serialized_pb=_b('\n\rmetrics.proto\"\x93\x01\n\x07Metrics\x12\x0c\n\x04unit\x18\x01 \x01(\t\x12\x13\n\x0bhttp_method\x18\x02 \x01(\t\x12\r\n\x05value\x18\x03 \x01(\x05\x12\x11\n\ttimestamp\x18\x04 \x01(\t\x12\x11\n\thttp_code\x18\x05 \x01(\t\x12\x0c\n\x04page\x18\x06 \x01(\t\x12\x12\n\nmetricType\x18\x07 \x01(\t\x12\x0e\n\x06server\x18\x08 \x01(\tb\x06proto3') +) +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + + + + +_METRICS = _descriptor.Descriptor( + name='Metrics', + full_name='Metrics', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='unit', full_name='Metrics.unit', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='http_method', full_name='Metrics.http_method', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='Metrics.value', index=2, + number=3, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='timestamp', full_name='Metrics.timestamp', index=3, + number=4, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='http_code', full_name='Metrics.http_code', index=4, + number=5, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='page', full_name='Metrics.page', index=5, + number=6, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='metricType', full_name='Metrics.metricType', index=6, + number=7, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='server', full_name='Metrics.server', index=7, + number=8, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=18, + serialized_end=165, +) + +DESCRIPTOR.message_types_by_name['Metrics'] = _METRICS + +Metrics = _reflection.GeneratedProtocolMessageType('Metrics', (_message.Message,), dict( + DESCRIPTOR = _METRICS, + __module__ = 'metrics_pb2' + # @@protoc_insertion_point(class_scope:Metrics) + )) +_sym_db.RegisterMessage(Metrics) + + +# @@protoc_insertion_point(module_scope) diff --git a/examples/quickstart/protobuf/pb_publisher.py b/examples/quickstart/protobuf/pb_publisher.py new file mode 100755 index 000000000000..73aeef2797b7 --- /dev/null +++ b/examples/quickstart/protobuf/pb_publisher.py @@ -0,0 +1,19 @@ +#!/usr/bin/env python + +import sys +import json + +from kafka import KafkaProducer +from metrics_pb2 import Metrics + + +producer = KafkaProducer(bootstrap_servers='localhost:9092') +topic = 'metrics_pb' + +for row in iter(sys.stdin): + d = json.loads(row) + metrics = Metrics() + for k, v in d.items(): + setattr(metrics, k, v) + pb = metrics.SerializeToString() + producer.send(topic, pb) diff --git a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 9333e58126ee..c7be90b8a2d2 100644 --- a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -43,6 +43,7 @@ import twitter4j.User; import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -113,7 +114,7 @@ public TwitterSpritzerFirehoseFactory( } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { final ConnectionLifeCycleListener connectionLifeCycleListener = new ConnectionLifeCycleListener() { @@ -213,6 +214,7 @@ public void onStallWarning(StallWarning warning) private final Runnable doNothingRunnable = new Runnable() { + @Override public void run() { } diff --git a/extendedset/src/main/java/io/druid/extendedset/intset/AbstractIntSet.java b/extendedset/src/main/java/io/druid/extendedset/intset/AbstractIntSet.java index 93066c918e70..6bd0e24153a8 100755 --- a/extendedset/src/main/java/io/druid/extendedset/intset/AbstractIntSet.java +++ b/extendedset/src/main/java/io/druid/extendedset/intset/AbstractIntSet.java @@ -114,7 +114,7 @@ public boolean containsAll(IntSet c) IntIterator itr = c.iterator(); boolean res = true; while (res && itr.hasNext()) { - res &= contains(itr.next()); + res = contains(itr.next()); } return res; } @@ -126,8 +126,7 @@ public boolean containsAll(IntSet c) public boolean containsAny(IntSet c) { IntIterator itr = c.iterator(); - boolean res = true; - while (res && itr.hasNext()) { + while (itr.hasNext()) { if (contains(itr.next())) { return true; } @@ -593,55 +592,6 @@ public int powerSetSize() return isEmpty() ? 0 : (int) Math.pow(2, size()) - 1; } - /** - * {@inheritDoc} - */ - @Override - public int powerSetSize(int min, int max) - { - if (min < 1 || max < min) { - throw new IllegalArgumentException(); - } - final int size = size(); - - // special cases - if (size < min) { - return 0; - } - if (size == min) { - return 1; - } - - /* - * Compute the sum of binomial coefficients ranging from (size choose - * max) to (size choose min) using dynamic programming - */ - - // trivial cases - max = Math.min(size, max); - if (max == min && (max == 0 || max == size)) { - return 1; - } - - // compute all binomial coefficients for "n" - int[] b = new int[size + 1]; - for (int i = 0; i <= size; i++) { - b[i] = 1; - } - for (int i = 1; i <= size; i++) { - for (int j = i - 1; j > 0; j--) { - b[j] += b[j - 1]; - } - } - - // sum binomial coefficients - int res = 0; - for (int i = min; i <= max; i++) { - res += b[i]; - } - return res; - } - /** * {@inheritDoc} */ diff --git a/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSet.java b/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSet.java index 208f02affc68..c96e01540f3b 100755 --- a/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSet.java +++ b/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSet.java @@ -41,9 +41,9 @@ * This class is an instance of {@link IntSet} internally represented by * compressed bitmaps though a RLE (Run-Length Encoding) compression algorithm. * See http - * ://ricerca.mat.uniroma3.it/users/colanton/docs/concise.pdf for more - * details. + * href="http://ricerca.mat.uniroma3.it/users/dipietro/publications/0020-0190.pdf"> + * http://ricerca.mat.uniroma3.it/users/dipietro/publications/0020-0190.pdf + * for more details. *

* Notice that the iterator by {@link #iterator()} is fail-fast, * similar to most {@link Collection}-derived classes. If the set is @@ -139,8 +139,8 @@ public ConciseSet(int[] words, boolean simulateWAH) * than a power of 2 takes at most O(lg(32)) time. The number of operations * is at most 12 + 9 * ceil(lg(32)). *

- * See http://graphics.stanford.edu/~seander/bithacks.html + * See + * http://graphics.stanford.edu/~seander/bithacks.html * * @param n number to divide * @@ -546,10 +546,10 @@ private void append(int i) */ private void appendLiteral(int word) { - // when we have a zero sequence of the maximum lenght (that is, + // when we have a zero sequence of the maximum length (that is, // 00.00000.1111111111111111111111111 = 0x01FFFFFF), it could happen // that we try to append a zero literal because the result of the given operation must be an - // empty set. Whitout the following test, we would have increased the + // empty set. Without the following test, we would have increased the // counter of the zero sequence, thus obtaining 0x02000000 that // represents a sequence with the first bit set! if (lastWordIndex == 0 && word == ConciseSetUtils.ALL_ZEROS_LITERAL && words[0] == 0x01FFFFFF) { @@ -751,8 +751,7 @@ private ConciseSet performOperation(ConciseSet other, Operator operator) break; case OR: res.last = Math.max(this.last, other.last); - invalidLast = false; - invalidLast |= thisItr.flush(res); + invalidLast = thisItr.flush(res); invalidLast |= otherItr.flush(res); break; case XOR: @@ -1324,7 +1323,7 @@ private boolean replaceWith(ConciseSet other) boolean isSimilar = (this.lastWordIndex == other.lastWordIndex) && (this.last == other.last); for (int i = 0; isSimilar && (i <= lastWordIndex); i++) { - isSimilar &= this.words[i] == other.words[i]; + isSimilar = this.words[i] == other.words[i]; } if (isSimilar) { @@ -1380,10 +1379,10 @@ public boolean add(int e) // -- If the literal is made up of all zeros, it definitely // cannot be part of a sequence (otherwise it would not have // been created). Thus, we can create a 1-bit literal word - // -- If there are MAX_LITERAL_LENGHT - 2 set bits, by adding + // -- If there are MAX_LITERAL_LENGTH - 2 set bits, by adding // the new one we potentially allow for a 1's sequence // together with the successive word - // -- If there are MAX_LITERAL_LENGHT - 1 set bits, by adding + // -- If there are MAX_LITERAL_LENGTH - 1 set bits, by adding // the new one we potentially allow for a 1's sequence // together with the successive and/or the preceding words if (!simulateWAH) { @@ -1616,10 +1615,7 @@ public boolean containsAll(IntSet c) // completely "covers" the second operator if (isSequenceWithNoBits(this.words[0]) && maxLiteralLengthMultiplication(getSequenceCount(this.words[0]) + 1) > other.last) { - if (isZeroSequence(this.words[0])) { - return false; - } - return true; + return !isZeroSequence(this.words[0]); } if (isSequenceWithNoBits(other.words[0]) && maxLiteralLengthMultiplication(getSequenceCount(other.words[0]) + 1) > this.last) { @@ -1702,17 +1698,11 @@ public boolean containsAny(IntSet c) // disjoint sets if (isSequenceWithNoBits(this.words[0]) && maxLiteralLengthMultiplication(getSequenceCount(this.words[0]) + 1) > other.last) { - if (isZeroSequence(this.words[0])) { - return false; - } - return true; + return !isZeroSequence(this.words[0]); } if (isSequenceWithNoBits(other.words[0]) && maxLiteralLengthMultiplication(getSequenceCount(other.words[0]) + 1) > this.last) { - if (isZeroSequence(other.words[0])) { - return false; - } - return true; + return !isZeroSequence(other.words[0]); } // scan "this" and "other" @@ -1791,17 +1781,11 @@ public boolean containsAtLeast(IntSet c, int minElements) // disjoint sets if (isSequenceWithNoBits(this.words[0]) && maxLiteralLengthMultiplication(getSequenceCount(this.words[0]) + 1) > other.last) { - if (isZeroSequence(this.words[0])) { - return false; - } - return true; + return !isZeroSequence(this.words[0]); } if (isSequenceWithNoBits(other.words[0]) && maxLiteralLengthMultiplication(getSequenceCount(other.words[0]) + 1) > this.last) { - if (isZeroSequence(other.words[0])) { - return false; - } - return true; + return !isZeroSequence(other.words[0]); } // resulting size diff --git a/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSetUtils.java b/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSetUtils.java index add784c904cd..6c3ece0797e1 100755 --- a/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSetUtils.java +++ b/extendedset/src/main/java/io/druid/extendedset/intset/ConciseSetUtils.java @@ -330,7 +330,7 @@ public static boolean isLiteralWithSingleOneBit(int word) public static int clearBitsAfterInLastWord(int lastWord, int lastSetBit) { - return lastWord &= ALL_ZEROS_LITERAL | (0xFFFFFFFF >>> (31 - lastSetBit)); + return lastWord & (ALL_ZEROS_LITERAL | (0xFFFFFFFF >>> (31 - lastSetBit))); } public static int onesUntil(int bit) diff --git a/extendedset/src/main/java/io/druid/extendedset/intset/ImmutableConciseSet.java b/extendedset/src/main/java/io/druid/extendedset/intset/ImmutableConciseSet.java index c2a54da2b9d7..34e82974a77a 100755 --- a/extendedset/src/main/java/io/druid/extendedset/intset/ImmutableConciseSet.java +++ b/extendedset/src/main/java/io/druid/extendedset/intset/ImmutableConciseSet.java @@ -39,7 +39,7 @@ public class ImmutableConciseSet { // Comparison is first by index, then one fills < literals < zero fills // one fills are sorted by length (longer one fills have priority) - // similarily, shorter zero fills have priority + // similarly, shorter zero fills have priority @Override public int compare(WordIterator i1, WordIterator i2) { @@ -82,7 +82,7 @@ public int compare(WordIterator i1, WordIterator i2) { // Comparison is first by index, then zero fills < literals < one fills // zero fills are sorted by length (longer zero fills have priority) - // similarily, shorter one fills have priority + // similarly, shorter one fills have priority @Override public int compare(WordIterator i1, WordIterator i2) { diff --git a/extendedset/src/main/java/io/druid/extendedset/intset/IntSet.java b/extendedset/src/main/java/io/druid/extendedset/intset/IntSet.java index a3a1f8bf961e..8100a6794e43 100755 --- a/extendedset/src/main/java/io/druid/extendedset/intset/IntSet.java +++ b/extendedset/src/main/java/io/druid/extendedset/intset/IntSet.java @@ -522,21 +522,6 @@ public interface IntSet extends Cloneable, Comparable */ public int powerSetSize(); - /** - * Computes the power-set size of the current set, composed by those subsets - * that have cardinality between min and max. - *

- * The returned power-set does not contain the empty set. - * - * @param min minimum subset size (greater than zero) - * @param max maximum subset size - * - * @return the power-set size - * - * @see #powerSet(int, int) - */ - public int powerSetSize(int min, int max); - /** * Computes the Jaccard similarity coefficient between this set and the * given set. @@ -605,6 +590,7 @@ public interface IntIterator extends org.roaringbitmap.IntIterator /** * @return true if the iterator has more elements. */ + @Override boolean hasNext(); /** @@ -612,6 +598,7 @@ public interface IntIterator extends org.roaringbitmap.IntIterator * * @throws NoSuchElementException iteration has no more elements. */ + @Override int next(); /** @@ -648,6 +635,7 @@ public interface IntIterator extends org.roaringbitmap.IntIterator * * @return a clone of the IntIterator */ + @Override public IntIterator clone(); } } diff --git a/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetTest.java b/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetTest.java index 2438bdad4e16..b2084b5f0aa1 100755 --- a/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetTest.java +++ b/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetTest.java @@ -25,6 +25,7 @@ import java.nio.IntBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.NoSuchElementException; @@ -1182,7 +1183,7 @@ public void testIntersection2() { final int[] ints1 = {33, 100000}; final int[] ints2 = {34, 100000}; - List expected = Arrays.asList(100000); + List expected = Collections.singletonList(100000); ConciseSet set1 = new ConciseSet(); for (int i : ints1) { @@ -1655,7 +1656,7 @@ public void testIntersectionFirstOneFillRemovedFromQueue() @Test public void testIntersectionTerminates() throws Exception { - verifyIntersection(Arrays.asList(), Arrays.asList(new ImmutableConciseSet(), new ImmutableConciseSet())); + verifyIntersection(Collections.emptyList(), Arrays.asList(new ImmutableConciseSet(), new ImmutableConciseSet())); } private void verifyIntersection(ConciseSet set1, ConciseSet set2) diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java index 2e41238fd1b1..98cd1a8ec4be 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/AzureBlob.java @@ -48,4 +48,13 @@ public String getContainer() { public String getPath() { return path; } + + @Override + public String toString() + { + return "AzureBlob{" + + "container=" + container + + ",path=" + path + + "}"; + } } diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index 233974cf22f8..c891c8472b0e 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -22,106 +22,71 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; -import io.druid.java.util.common.logger.Logger; import io.druid.storage.azure.AzureByteSource; import io.druid.storage.azure.AzureStorage; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Iterator; -import java.util.LinkedList; +import java.util.Collection; import java.util.List; /** * This class is heavily inspired by the StaticS3FirehoseFactory class in the io.druid.firehose.s3 package */ -public class StaticAzureBlobStoreFirehoseFactory implements FirehoseFactory { - private static final Logger log = new Logger(StaticAzureBlobStoreFirehoseFactory.class); - +public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory +{ private final AzureStorage azureStorage; private final List blobs; @JsonCreator public StaticAzureBlobStoreFirehoseFactory( @JacksonInject("azureStorage") AzureStorage azureStorage, - @JsonProperty("blobs") AzureBlob[] blobs - ) { + @JsonProperty("blobs") List blobs, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Long fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry + ) + { + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + this.blobs = blobs; this.azureStorage = azureStorage; - this.blobs = ImmutableList.copyOf(blobs); } @JsonProperty - public List getBlobs() { + public List getBlobs() + { return blobs; } @Override - public Firehose connect(StringInputRowParser stringInputRowParser) throws IOException { - Preconditions.checkNotNull(azureStorage, "null azureStorage"); - - final LinkedList objectQueue = Lists.newLinkedList(blobs); - - return new FileIteratingFirehose( - new Iterator() { - @Override - public boolean hasNext() { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() { - final AzureBlob nextURI = objectQueue.poll(); - - final String container = nextURI.getContainer(); - final String path = nextURI.getPath().startsWith("/") - ? nextURI.getPath().substring(1) - : nextURI.getPath(); - - try { - final InputStream innerInputStream = new AzureByteSource(azureStorage, container, path).openStream(); - + protected Collection initObjects() + { + return blobs; + } - final InputStream outerInputStream = path.endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; + @Override + protected InputStream openObjectStream(AzureBlob object) throws IOException + { + return makeByteSource(azureStorage, object).openStream(); + } - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8) - ) - ); - } catch (Exception e) { - log.error(e, - "Exception opening container[%s] blob[%s]", - container, - path - ); + @Override + protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + } - throw Throwables.propagate(e); - } - } + private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) + { + final String container = object.getContainer(); + final String path = object.getPath().startsWith("/") + ? object.getPath().substring(1) + : object.getPath(); - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }, - stringInputRowParser - ); + return new AzureByteSource(azureStorage, container, path); } } diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java index 5e42b951779d..0b9f7bea9afa 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPusher.java @@ -29,12 +29,12 @@ import io.druid.java.util.common.logger.Logger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.net.URI; import java.net.URISyntaxException; import java.util.Map; import java.util.concurrent.Callable; @@ -85,7 +85,7 @@ public File createSegmentDescriptorFile(final ObjectMapper jsonMapper, final Dat public Map getAzurePaths(final DataSegment segment) { - final String storageDir = DataSegmentPusherUtil.getStorageDir(segment); + final String storageDir = this.getStorageDir(segment); return ImmutableMap.of( "index", String.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME), @@ -109,16 +109,7 @@ public DataSegment uploadDataSegment( final DataSegment outSegment = segment .withSize(size) - .withLoadSpec( - ImmutableMap.of( - "type", - AzureStorageDruidModule.SCHEME, - "containerName", - config.getContainer(), - "blobPath", - azurePaths.get("index") - ) - ) + .withLoadSpec(this.makeLoadSpec(new URI(azurePaths.get("index")))) .withBinaryVersion(version); log.info("Deleting file [%s]", compressedSegmentData); @@ -174,4 +165,17 @@ public DataSegment call() throws Exception } } } + + @Override + public Map makeLoadSpec(URI uri) + { + return ImmutableMap.of( + "type", + AzureStorageDruidModule.SCHEME, + "containerName", + config.getContainer(), + "blobPath", + uri.toString() + ); + } } diff --git a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java index 6cb9680ecbb2..e5913c50f5e1 100644 --- a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java +++ b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPusherTest.java @@ -27,7 +27,6 @@ import com.microsoft.azure.storage.StorageException; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.MapUtils; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMockSupport; @@ -112,9 +111,9 @@ public void testPush() throws Exception @Test public void getAzurePathsTest() { - final String storageDir = DataSegmentPusherUtil.getStorageDir(dataSegment); - AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, jsonMapper); + AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, jsonMapper); + final String storageDir = pusher.getStorageDir(dataSegment); Map paths = pusher.getAzurePaths(dataSegment); assertEquals(String.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME), paths.get("index")); diff --git a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java index 973de4297555..b26253e03ed1 100644 --- a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java +++ b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java @@ -30,12 +30,13 @@ import io.druid.java.util.common.logger.Logger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import java.io.File; import java.io.FileInputStream; import java.io.IOException; +import java.net.URI; +import java.util.Map; /** * Cassandra Segment Pusher @@ -77,7 +78,7 @@ public DataSegment push(final File indexFilesDir, DataSegment segment) throws IO log.info("Writing [%s] to C*", indexFilesDir); String key = JOINER.join( config.getKeyspace().isEmpty() ? null : config.getKeyspace(), - DataSegmentPusherUtil.getStorageDir(segment) + this.getStorageDir(segment) ); // Create index @@ -114,4 +115,10 @@ ImmutableMap. of("type", "c*", "key", key) compressedIndexFile.delete(); return segment; } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException("not supported"); + } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java index fc25ba9b6767..fb013ed55a74 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/CloudFilesBlob.java @@ -62,4 +62,14 @@ public String getRegion() { return region; } + + @Override + public String toString() + { + return "CloudFilesBlob{" + + "container=" + container + + ",path=" + path + + ",region=" + region + + "}"; + } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index c3f5d23ab999..29ecfb3bbc8a 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -22,34 +22,19 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; -import io.druid.java.util.common.parsers.ParseException; import io.druid.storage.cloudfiles.CloudFilesByteSource; import io.druid.storage.cloudfiles.CloudFilesObjectApiProxy; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Iterator; -import java.util.LinkedList; +import java.util.Collection; import java.util.List; -public class StaticCloudFilesFirehoseFactory implements FirehoseFactory +public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class); @@ -59,11 +44,17 @@ public class StaticCloudFilesFirehoseFactory implements FirehoseFactory blobs, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Long fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.cloudFilesApi = cloudFilesApi; - this.blobs = ImmutableList.copyOf(blobs); + this.blobs = blobs; } @JsonProperty @@ -73,67 +64,31 @@ public List getBlobs() } @Override - public Firehose connect(StringInputRowParser stringInputRowParser) throws IOException, ParseException + protected Collection initObjects() { - Preconditions.checkNotNull(cloudFilesApi, "null cloudFilesApi"); - - final LinkedList objectQueue = Lists.newLinkedList(blobs); - - return new FileIteratingFirehose( - new Iterator() - { - - @Override - public boolean hasNext() - { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() - { - final CloudFilesBlob nextURI = objectQueue.poll(); - - final String region = nextURI.getRegion(); - final String container = nextURI.getContainer(); - final String path = nextURI.getPath(); - - log.info("Retrieving file from region[%s], container[%s] and path [%s]", - region, container, path - ); - CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy( - cloudFilesApi, region, container); - final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path); - - try { - final InputStream innerInputStream = byteSource.openStream(); - final InputStream outerInputStream = path.endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; - - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8))); - } - catch (IOException e) { - log.error(e, - "Exception opening container[%s] blob[%s] from region[%s]", - container, path, region - ); - - throw Throwables.propagate(e); - } - } + return blobs; + } - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } + @Override + protected InputStream openObjectStream(CloudFilesBlob object) throws IOException + { + final String region = object.getRegion(); + final String container = object.getContainer(); + final String path = object.getPath(); - }, - stringInputRowParser + log.info("Retrieving file from region[%s], container[%s] and path [%s]", + region, container, path ); + CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy( + cloudFilesApi, region, container); + final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path); + + return byteSource.openStream(); } + @Override + protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java index fbef1bfe01f5..a17a6872b586 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java @@ -34,6 +34,8 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.net.URI; +import java.util.Map; import java.util.concurrent.Callable; public class CloudFilesDataSegmentPusher implements DataSegmentPusher @@ -75,7 +77,7 @@ public String getPathForHadoop(final String dataSource) @Override public DataSegment push(final File indexFilesDir, final DataSegment inSegment) throws IOException { - final String segmentPath = CloudFilesUtils.buildCloudFilesPath(this.config.getBasePath(), inSegment); + final String segmentPath = CloudFilesUtils.buildCloudFilesPath(this.config.getBasePath(), getStorageDir(inSegment)); File descriptorFile = null; File zipOutFile = null; @@ -112,18 +114,7 @@ public DataSegment call() throws Exception final DataSegment outSegment = inSegment .withSize(indexSize) - .withLoadSpec( - ImmutableMap.of( - "type", - CloudFilesStorageDruidModule.SCHEME, - "region", - segmentData.getRegion(), - "container", - segmentData.getContainer(), - "path", - segmentData.getPath() - ) - ) + .withLoadSpec(makeLoadSpec(new URI(segmentData.getPath()))) .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir)); return outSegment; @@ -146,4 +137,19 @@ public DataSegment call() throws Exception } } } + + @Override + public Map makeLoadSpec(URI uri) + { + return ImmutableMap.of( + "type", + CloudFilesStorageDruidModule.SCHEME, + "region", + objectApi.getRegion(), + "container", + objectApi.getContainer(), + "path", + uri.toString() + ); + } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesUtils.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesUtils.java index 31b4cafdf383..e409964de399 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesUtils.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesUtils.java @@ -22,8 +22,6 @@ import com.google.common.base.Predicate; import io.druid.java.util.common.RetryUtils; -import io.druid.segment.loading.DataSegmentPusherUtil; -import io.druid.timeline.DataSegment; import java.io.IOException; import java.util.concurrent.Callable; @@ -70,9 +68,4 @@ public static String buildCloudFilesPath(String basePath, final String fileName) return path; } - public static String buildCloudFilesPath(String basePath, final DataSegment segment) - { - return buildCloudFilesPath(basePath, DataSegmentPusherUtil.getStorageDir(segment)); - } - } diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java index 532af26dee2d..a9e80b0762d2 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java @@ -37,6 +37,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -168,7 +169,7 @@ public String getName() @Override public List requiredFields() { - return Arrays.asList(fieldName); + return Collections.singletonList(fieldName); } @Override diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java index 62e95a2c0936..25096ab27b64 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java @@ -77,5 +77,6 @@ public void close() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index 08b1a8bc7a46..fbda378b0108 100644 --- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; - import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; @@ -40,7 +39,7 @@ import org.joda.time.DateTime; import org.junit.Test; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class DistinctCountTimeseriesQueryTest @@ -100,7 +99,7 @@ public void testTopNWithDistinctCountAgg() throws Exception Lists.>newLinkedList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( time, new TimeseriesResultValue( diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index b6ebb11d8bd2..9a82a171d17c 100644 --- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -44,6 +44,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -118,7 +119,7 @@ public ByteBuffer get() Lists.>newLinkedList() ); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result<>( time, new TopNResultValue( diff --git a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java index 94b7bb79cde8..1bd78f8a3027 100644 --- a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java +++ b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java @@ -38,10 +38,10 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.java.util.common.collect.JavaCompatUtils; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -127,7 +127,7 @@ public RocketMQFirehoseFactory( private boolean hasMessagesPending() { - for (ConcurrentHashMap.Entry> entry : messageQueueTreeSetMap.entrySet()) { + for (Map.Entry> entry : messageQueueTreeSetMap.entrySet()) { if (!entry.getValue().isEmpty()) { return true; } @@ -137,7 +137,10 @@ private boolean hasMessagesPending() } @Override - public Firehose connect(ByteBufferInputRowParser byteBufferInputRowParser) throws IOException, ParseException + public Firehose connect( + ByteBufferInputRowParser byteBufferInputRowParser, + File temporaryDirectory + ) throws IOException, ParseException { Set newDimExclus = Sets.union( @@ -202,8 +205,8 @@ public boolean hasMore() for (Map.Entry> entry : topicQueueMap.entrySet()) { for (MessageQueue messageQueue : entry.getValue()) { - if (JavaCompatUtils.keySet(messageQueueTreeSetMap).contains(messageQueue) - && !messageQueueTreeSetMap.get(messageQueue).isEmpty()) { + ConcurrentSkipListSet messages = messageQueueTreeSetMap.get(messageQueue); + if (messages != null && !messages.isEmpty()) { hasMore = true; } else { try { @@ -251,10 +254,9 @@ public InputRow nextRow() MessageExt message = entry.getValue().pollFirst(); InputRow inputRow = theParser.parse(ByteBuffer.wrap(message.getBody())); - if (!JavaCompatUtils.keySet(windows).contains(entry.getKey())) { - windows.put(entry.getKey(), new ConcurrentSkipListSet()); - } - windows.get(entry.getKey()).add(message.getQueueOffset()); + windows + .computeIfAbsent(entry.getKey(), k -> new ConcurrentSkipListSet<>()) + .add(message.getQueueOffset()); return inputRow; } } @@ -308,7 +310,7 @@ public void close() throws IOException /** * Pull request. */ - final class DruidPullRequest + static final class DruidPullRequest { private final MessageQueue messageQueue; private final String tag; @@ -434,13 +436,9 @@ private void doPull() switch (pullResult.getPullStatus()) { case FOUND: // Handle pull result. - if (!JavaCompatUtils.keySet(messageQueueTreeSetMap).contains(pullRequest.getMessageQueue())) { - messageQueueTreeSetMap.putIfAbsent( - pullRequest.getMessageQueue(), - new ConcurrentSkipListSet<>(new MessageComparator()) - ); - } - messageQueueTreeSetMap.get(pullRequest.getMessageQueue()).addAll(pullResult.getMsgFoundList()); + messageQueueTreeSetMap + .computeIfAbsent(pullRequest.getMessageQueue(), k -> new ConcurrentSkipListSet<>(MESSAGE_COMPARATOR)) + .addAll(pullResult.getMsgFoundList()); break; case NO_NEW_MSG: @@ -508,14 +506,7 @@ protected void onWaitEnd() /** * Compare messages pulled from same message queue according to queue offset. */ - static final class MessageComparator implements Comparator - { - @Override - public int compare(MessageExt lhs, MessageExt rhs) - { - return Long.compare(lhs.getQueueOffset(), lhs.getQueueOffset()); - } - } + private static final Comparator MESSAGE_COMPARATOR = Comparator.comparingLong(MessageExt::getQueueOffset); /** diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java index 0b6b6cc6eba0..d0109973d87b 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/GoogleBlob.java @@ -41,5 +41,14 @@ public String getBucket() { public String getPath() { return path; } + + @Override + public String toString() + { + return "GoogleBlob {" + + "bucket=" + bucket + + ",path=" + path + + "}"; + } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 907571d58ffa..4f8afca76ca3 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -22,43 +22,35 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; -import io.druid.java.util.common.logger.Logger; import io.druid.storage.google.GoogleByteSource; import io.druid.storage.google.GoogleStorage; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Iterator; -import java.util.LinkedList; +import java.util.Collection; import java.util.List; -public class StaticGoogleBlobStoreFirehoseFactory implements FirehoseFactory { - private static final Logger LOG = new Logger(StaticGoogleBlobStoreFirehoseFactory.class); - +public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory +{ private final GoogleStorage storage; private final List blobs; @JsonCreator public StaticGoogleBlobStoreFirehoseFactory( @JacksonInject GoogleStorage storage, - @JsonProperty("blobs") GoogleBlob[] blobs - ) { + @JsonProperty("blobs") List blobs, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Long fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry + ) + { + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); this.storage = storage; - this.blobs = ImmutableList.copyOf(blobs); + this.blobs = blobs; } @JsonProperty @@ -67,57 +59,26 @@ public List getBlobs() { } @Override - public Firehose connect(StringInputRowParser stringInputRowParser) throws IOException { - Preconditions.checkNotNull(storage, "null storage"); - - final LinkedList objectQueue = Lists.newLinkedList(blobs); - - return new FileIteratingFirehose( - new Iterator() { - @Override - public boolean hasNext() { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() { - final GoogleBlob nextURI = objectQueue.poll(); - - final String bucket = nextURI.getBucket(); - final String path = nextURI.getPath().startsWith("/") - ? nextURI.getPath().substring(1) - : nextURI.getPath(); - - try { - final InputStream innerInputStream = new GoogleByteSource(storage, bucket, path).openStream(); - - final InputStream outerInputStream = path.endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; - - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8) - ) - ); - } catch (Exception e) { - LOG.error(e, - "Exception opening bucket[%s] blob[%s]", - bucket, - path - ); + protected Collection initObjects() + { + return blobs; + } - throw Throwables.propagate(e); - } - } + @Override + protected InputStream openObjectStream(GoogleBlob object) throws IOException + { + final String bucket = object.getBucket(); + final String path = object.getPath().startsWith("/") + ? object.getPath().substring(1) + : object.getPath(); + + return new GoogleByteSource(storage, bucket, path).openStream(); + } - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }, - stringInputRowParser - ); + @Override + protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java index 8bd9fbb8ee7c..d94f233c81f1 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPusher.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.http.InputStreamContent; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; @@ -28,13 +30,14 @@ import io.druid.java.util.common.logger.Logger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.net.URI; +import java.util.Map; public class GoogleDataSegmentPusher implements DataSegmentPusher { @@ -82,7 +85,8 @@ public File createDescriptorFile(final ObjectMapper jsonMapper, final DataSegmen return descriptorFile; } - public void insert(final File file, final String contentType, final String path) throws IOException { + public void insert(final File file, final String contentType, final String path) throws IOException + { LOG.info("Inserting [%s] to [%s]", file, path); FileInputStream fileSteam = new FileInputStream(file); @@ -105,19 +109,13 @@ public DataSegment push(final File indexFilesDir, final DataSegment segment) thr try { indexFile = File.createTempFile("index", ".zip"); final long indexSize = CompressionUtils.zip(indexFilesDir, indexFile); - final String storageDir = DataSegmentPusherUtil.getStorageDir(segment); + final String storageDir = this.getStorageDir(segment); final String indexPath = buildPath(storageDir + "/" + "index.zip"); final String descriptorPath = buildPath(storageDir + "/" + "descriptor.json"); final DataSegment outSegment = segment .withSize(indexSize) - .withLoadSpec( - ImmutableMap.of( - "type", GoogleStorageDruidModule.SCHEME, - "bucket", config.getBucket(), - "path", indexPath - ) - ) + .withLoadSpec(makeLoadSpec(config.getBucket(), indexPath)) .withBinaryVersion(version); descriptorFile = createDescriptorFile(jsonMapper, outSegment); @@ -129,7 +127,8 @@ public DataSegment push(final File indexFilesDir, final DataSegment segment) thr } catch (Exception e) { throw Throwables.propagate(e); - } finally { + } + finally { if (indexFile != null) { LOG.info("Deleting file [%s]", indexFile); indexFile.delete(); @@ -142,12 +141,29 @@ public DataSegment push(final File indexFilesDir, final DataSegment segment) thr } } - public String buildPath(final String path) + @VisibleForTesting + String buildPath(final String path) { - if (config.getPrefix() != "") { + if (!Strings.isNullOrEmpty(config.getPrefix())) { return config.getPrefix() + "/" + path; } else { return path; } } + + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + // remove the leading "/" + return makeLoadSpec(config.getBucket(),finalIndexZipFilePath.getPath().substring(1)); + } + + private Map makeLoadSpec(String bucket, String path) { + return ImmutableMap.of( + "type", GoogleStorageDruidModule.SCHEME, + "bucket", bucket, + "path", path + ); + } + } diff --git a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java index 862974c513ba..83d35601ff18 100644 --- a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java +++ b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.Maps; import com.google.common.io.Files; import io.druid.jackson.DefaultObjectMapper; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -106,7 +105,7 @@ public void testPush() throws Exception jsonMapper ).addMockedMethod("insert", File.class, String.class, String.class).createMock(); - final String storageDir = DataSegmentPusherUtil.getStorageDir(segmentToPush); + final String storageDir = pusher.getStorageDir(segmentToPush); final String indexPath = prefix + "/" + storageDir + "/" + "index.zip"; final String descriptorPath = prefix + "/" + storageDir + "/" + "descriptor.json"; @@ -132,4 +131,22 @@ public void testPush() throws Exception verifyAll(); } + + @Test + public void testBuildPath() + { + GoogleAccountConfig config = new GoogleAccountConfig(); + StringBuilder sb = new StringBuilder(); + sb.setLength(0); + config.setPrefix(sb.toString()); // avoid cached empty string + GoogleDataSegmentPusher pusher = new GoogleDataSegmentPusher( + storage, + config, + jsonMapper + ); + Assert.assertEquals("/path", pusher.buildPath("/path")); + config.setPrefix(null); + Assert.assertEquals("/path", pusher.buildPath("/path")); + } + } diff --git a/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcExtensionsModule.java b/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcExtensionsModule.java index 9619c884e1f6..92d51a56ca75 100644 --- a/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcExtensionsModule.java +++ b/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcExtensionsModule.java @@ -24,14 +24,14 @@ import com.google.inject.Binder; import io.druid.initialization.DruidModule; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class OrcExtensionsModule implements DruidModule { @Override public List getJacksonModules() { - return Arrays.asList( + return Collections.singletonList( new SimpleModule("OrcInputRowParserModule") .registerSubtypes( new NamedType(OrcHadoopInputRowParser.class, "orc") diff --git a/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcHadoopInputRowParser.java index 1a2a4d09730f..93785e4ed7ff 100644 --- a/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcHadoopInputRowParser.java +++ b/extensions-contrib/orc-extensions/src/main/java/io/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -70,6 +70,7 @@ public OrcHadoopInputRowParser( initialize(); } + @SuppressWarnings("ArgumentParameterSwap") @Override public InputRow parse(OrcStruct input) { diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java index f56479cef5b6..949247d9a294 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -216,7 +216,8 @@ public void setUp() throws Exception true, null, false, - false + false, + null ) ) ); diff --git a/extensions-contrib/parquet-extensions/src/main/java/io/druid/data/input/parquet/ParquetExtensionsModule.java b/extensions-contrib/parquet-extensions/src/main/java/io/druid/data/input/parquet/ParquetExtensionsModule.java index 657a94ae1965..12f0b88b772e 100644 --- a/extensions-contrib/parquet-extensions/src/main/java/io/druid/data/input/parquet/ParquetExtensionsModule.java +++ b/extensions-contrib/parquet-extensions/src/main/java/io/druid/data/input/parquet/ParquetExtensionsModule.java @@ -25,7 +25,7 @@ import com.google.inject.Binder; import io.druid.initialization.DruidModule; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class ParquetExtensionsModule implements DruidModule @@ -34,8 +34,8 @@ public class ParquetExtensionsModule implements DruidModule @Override public List getJacksonModules() { - return Arrays.asList( - new SimpleModule("ParuqetInputRowParserModule") + return Collections.singletonList( + new SimpleModule("ParquetInputRowParserModule") .registerSubtypes( new NamedType(ParquetHadoopInputRowParser.class, "parquet") ) diff --git a/extensions-contrib/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetReadSupport.java b/extensions-contrib/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetReadSupport.java index a8e564bd91df..f4f17f9e279a 100755 --- a/extensions-contrib/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetReadSupport.java +++ b/extensions-contrib/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetReadSupport.java @@ -73,6 +73,7 @@ private MessageType getPartialReadSchema(InitContext context) return new MessageType(name, partialFields); } + @Override public ReadContext init(InitContext context) { MessageType requestedProjection = getSchemaForRead(context.getFileSchema(), getPartialReadSchema(context)); diff --git a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index b48083fe2991..66c91770a2b7 100644 --- a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -41,6 +41,7 @@ import net.jodah.lyra.retry.RetryPolicy; import net.jodah.lyra.util.Duration; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.BlockingQueue; @@ -134,7 +135,7 @@ public JacksonifiedConnectionFactory getConnectionFactory() } @Override - public Firehose connect(final ByteBufferInputRowParser firehoseParser) throws IOException + public Firehose connect(final ByteBufferInputRowParser firehoseParser, File temporaryDirectory) throws IOException { ConnectionOptions lyraOptions = new ConnectionOptions(this.connectionFactory); Config lyraConfig = new Config() diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java index a526868c3639..1606ded6156e 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java @@ -191,15 +191,15 @@ public int hashCode() public String toString() { return "ScanQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", descending=" + isDescending() + - ", resultFormat='" + resultFormat + '\'' + - ", batchSize=" + batchSize + - ", limit=" + limit + - ", dimFilter=" + dimFilter + - ", columns=" + columns + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", descending=" + isDescending() + + ", resultFormat='" + resultFormat + '\'' + + ", batchSize=" + batchSize + + ", limit=" + limit + + ", dimFilter=" + dimFilter + + ", columns=" + columns + + '}'; } /** diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java index f33ea997b268..4a1633aaa388 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java @@ -30,6 +30,7 @@ import java.util.List; public class ScanQueryDruidModule implements DruidModule { + @Override public void configure(Binder binder) { DruidBinders.queryToolChestBinder(binder) .addBinding(ScanQuery.class) @@ -42,6 +43,7 @@ public void configure(Binder binder) { .in(LazySingleton.class); } + @Override public List getJacksonModules() { return Arrays.asList( new SimpleModule("ScanQueryDruidModule") diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java index f102d1c3ec67..d2f07a17da31 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java @@ -22,6 +22,7 @@ import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import java.io.IOException; @@ -36,13 +37,15 @@ public class ScanQueryLimitRowIterator implements CloseableIterator baseRunner, ScanQuery query, + QueryRunner baseRunner, + QueryPlus queryPlus, Map responseContext ) { + ScanQuery query = (ScanQuery) queryPlus.getQuery(); resultFormat = query.getResultFormat(); limit = query.getLimit(); - Sequence baseSequence = baseRunner.run(query, responseContext); + Sequence baseSequence = baseRunner.run(queryPlus, responseContext); yielder = baseSequence.toYielder( null, new YieldingAccumulator() diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java index 8cc1c7e3ffce..26edb06c9fd4 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java @@ -25,9 +25,10 @@ import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.GenericQueryMetricsFactory; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.MetricManipulationFn; @@ -55,12 +56,12 @@ public QueryRunner mergeResults(final QueryRunner run( - final Query query, final Map responseContext + final QueryPlus queryPlus, final Map responseContext ) { - ScanQuery scanQuery = (ScanQuery) query; + ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); if (scanQuery.getLimit() == Long.MAX_VALUE) { - return runner.run(query, responseContext); + return runner.run(queryPlus, responseContext); } return new BaseSequence<>( new BaseSequence.IteratorMaker() @@ -68,7 +69,7 @@ public Sequence run( @Override public ScanQueryLimitRowIterator make() { - return new ScanQueryLimitRowIterator(runner, (ScanQuery) query, responseContext); + return new ScanQueryLimitRowIterator(runner, queryPlus, responseContext); } @Override @@ -109,14 +110,15 @@ public QueryRunner preMergeQueryDecoration(final QueryRunner run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { - ScanQuery scanQuery = (ScanQuery) query; + ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); if (scanQuery.getDimensionsFilter() != null) { scanQuery = scanQuery.withDimFilter(scanQuery.getDimensionsFilter().optimize()); + queryPlus = queryPlus.withQuery(scanQuery); } - return runner.run(scanQuery, responseContext); + return runner.run(queryPlus, responseContext); } }; } diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java index fcfe11b43664..e520f981a0ec 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java @@ -26,6 +26,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.Query; import io.druid.query.QueryContexts; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -70,12 +71,12 @@ public QueryRunner mergeRunners( { @Override public Sequence run( - final Query query, final Map responseContext + final QueryPlus queryPlus, final Map responseContext ) { // Note: this variable is effective only when queryContext has a timeout. // See the comment of CTX_TIMEOUT_AT. - final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(query); + final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); return Sequences.concat( Sequences.map( @@ -85,7 +86,7 @@ public Sequence run( @Override public Sequence apply(final QueryRunner input) { - return input.run(query, responseContext); + return input.run(queryPlus, responseContext); } } ) @@ -100,7 +101,7 @@ public QueryToolChest getToolchest() return toolChest; } - private class ScanQueryRunner implements QueryRunner + private static class ScanQueryRunner implements QueryRunner { private final ScanQueryEngine engine; private final Segment segment; @@ -113,9 +114,10 @@ public ScanQueryRunner(ScanQueryEngine engine, Segment segment) @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { + Query query = queryPlus.getQuery(); if (!(query instanceof ScanQuery)) { throw new ISE("Got a [%s] which isn't a %s", query.getClass(), ScanQuery.class); } diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java index 8f088ad8a687..254f4dceda96 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java @@ -28,7 +28,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.DefaultGenericQueryMetricsFactory; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -214,15 +214,15 @@ public void testMergeResultsWithLimit() new QueryRunner() { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate results back from 2 historicals List> sequences = Lists.newArrayListWithExpectedSize(2); - sequences.add(factory.createRunner(segment0).run(query, new HashMap())); - sequences.add(factory.createRunner(segment1).run(query, new HashMap())); + sequences.add(factory.createRunner(segment0).run(queryPlus, new HashMap())); + sequences.add(factory.createRunner(segment1).run(queryPlus, new HashMap())); return new MergeSequence<>( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple(sequences) ); } diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java index ceeae9e52516..b296534a87cf 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java @@ -47,6 +47,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; @@ -414,8 +415,7 @@ public void testFullSelectNoResults() Lists.newArrayList() ); - List expectedResults = Arrays.asList( - ); + List expectedResults = Collections.emptyList(); verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); } diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index 0e2a2969c969..36d11e426f3a 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -96,4 +96,24 @@ + + + + strict + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + -Xep:MissingOverride:WARN + + + + + + + diff --git a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java index 4deed399a200..668e3e4cd995 100644 --- a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java +++ b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java @@ -25,7 +25,7 @@ import com.google.inject.Binder; import io.druid.initialization.DruidModule; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class ThriftExtensionsModule implements DruidModule @@ -34,7 +34,7 @@ public class ThriftExtensionsModule implements DruidModule @Override public List getJacksonModules() { - return Arrays.asList( + return Collections.singletonList( new SimpleModule("ThriftInputRowParserModule") .registerSubtypes( new NamedType(ThriftInputRowParser.class, "thrift") diff --git a/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampAggregatorFactory.java b/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampAggregatorFactory.java index c97e313be997..15fd5d5eef59 100644 --- a/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampAggregatorFactory.java +++ b/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampAggregatorFactory.java @@ -29,6 +29,7 @@ import java.nio.ByteBuffer; import java.sql.Timestamp; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -142,7 +143,7 @@ public String getTimeFormat() @Override public List requiredFields() { - return Arrays.asList(fieldName); + return Collections.singletonList(fieldName); } @Override diff --git a/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampMinMaxModule.java b/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampMinMaxModule.java index 3d0204b85585..1cf15cf964b7 100644 --- a/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampMinMaxModule.java +++ b/extensions-contrib/time-min-max/src/main/java/io/druid/query/aggregation/TimestampMinMaxModule.java @@ -24,7 +24,7 @@ import com.google.inject.Binder; import io.druid.initialization.DruidModule; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class TimestampMinMaxModule implements DruidModule @@ -32,7 +32,7 @@ public class TimestampMinMaxModule implements DruidModule @Override public List getJacksonModules() { - return Arrays.asList( + return Collections.singletonList( new SimpleModule("TimestampMinMaxModule") .registerSubtypes( new NamedType(TimestampMaxAggregatorFactory.class, "timeMax"), diff --git a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java index 33ce97c231ee..581561d01903 100644 --- a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java +++ b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java @@ -226,6 +226,7 @@ public String getValueDimension() return valueDimension; } + @Override @JsonProperty public String getOutputName() { diff --git a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java index 46d99cbbcc96..11ef3c9fea8d 100644 --- a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java @@ -55,6 +55,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -96,7 +97,9 @@ public static Iterable constructorFeeder() throws IOException new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("dim", "keys", "values")), null, null), "\t", ",", - Arrays.asList("ts", "dim", "keys", "values") + Arrays.asList("ts", "dim", "keys", "values"), + false, + 0 ) , "utf8" ); @@ -162,8 +165,8 @@ public void testBasic() throws Exception "params", mapOf("key1", "value1", "key5", "value5") ) ); - List virtualColumns = Arrays.asList(new MapVirtualColumn("keys", "values", "params")); - SelectQuery selectQuery = builder.dimensions(Arrays.asList("dim")) + List virtualColumns = Collections.singletonList(new MapVirtualColumn("keys", "values", "params")); + SelectQuery selectQuery = builder.dimensions(Collections.singletonList("dim")) .metrics(Arrays.asList("params.key1", "params.key3", "params.key5", "params")) .virtualColumns(virtualColumns) .build(); diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index 2516ea9e8261..a02d043223d6 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -79,6 +79,12 @@ io.confluent kafka-schema-registry-client ${confluent.version} + + + org.slf4j + slf4j-log4j12 + + org.apache.hadoop @@ -134,4 +140,24 @@ + + + + strict + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + -Xep:MissingOverride:WARN + + + + + + + diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java index 40d422287e92..452eb6d46534 100644 --- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java +++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java @@ -55,7 +55,7 @@ public RecordReader createRecordReader( if (StringUtils.isNotBlank(schemaFilePath)) { log.info("Using file: %s as reader schema.", schemaFilePath); try (FSDataInputStream inputStream = - FileSystem.get(context.getConfiguration()).open(new Path(schemaFilePath))) { + FileSystem.get(context.getConfiguration()).open(new Path(schemaFilePath))) { readerSchema = new Schema.Parser().parse(inputStream); } } diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java index 1cebd3f1fee4..f62bcb2df115 100644 --- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java +++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java @@ -80,14 +80,12 @@ public GenericRecord parse(ByteBuffer bytes) DatumReader reader = new GenericDatumReader<>(schema); try (ByteBufferInputStream inputStream = new ByteBufferInputStream(Collections.singletonList(bytes))) { return reader.read(null, DecoderFactory.get().binaryDecoder(inputStream, null)); - } - catch (EOFException eof) { + } catch (EOFException eof) { // waiting for avro v1.9.0 (#AVRO-813) throw new ParseException( eof, "Avro's unnecessary EOFException, detail: [%s]", "https://issues.apache.org/jira/browse/AVRO-813" ); - } - catch (IOException e) { + } catch (IOException e) { throw new ParseException(e, "Fail to decode avro message!"); } } @@ -110,8 +108,8 @@ public boolean equals(Object o) return false; } return !(schemaRepository != null - ? !schemaRepository.equals(that.schemaRepository) - : that.schemaRepository != null); + ? !schemaRepository.equals(that.schemaRepository) + : that.schemaRepository != null); } @Override diff --git a/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java index c83682142d6a..e8d24e76012e 100644 --- a/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java @@ -109,6 +109,7 @@ public class AvroStreamInputRowParserTest public static final ByteBuffer SOME_BYTES_VALUE = ByteBuffer.allocate(8); private static final Function TO_STRING_INCLUDING_NULL = new Function() { + @Override public String apply(Object o) { return String.valueOf(o); } }; diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java index 2014314675c7..d97622d5ff83 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java @@ -73,5 +73,6 @@ public void close() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java index 3fee2f0467a9..1babde4d8060 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java @@ -105,7 +105,7 @@ public boolean equals(Object obj) if (estimate != that.estimate || highBound != that.highBound || lowBound != that.lowBound || - numStdDev != numStdDev) { + numStdDev != that.numStdDev) { return false; } return true; diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index c7d8429e2764..8dd014629b99 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -140,12 +140,17 @@ emitter provided + + org.apache.hadoop + hadoop-aws + ${hadoop.compile.version} + commons-io commons-io provided - + junit @@ -178,6 +183,12 @@ ${hadoop.compile.version} test - + + io.druid + druid-indexing-hadoop + ${project.parent.version} + test + + diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java index 6b80e3619c25..25dcf4fb4f9b 100644 --- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java +++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -29,17 +29,19 @@ import io.druid.java.util.common.logger.Logger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.HadoopFsWrapper; import org.apache.hadoop.fs.Path; +import org.joda.time.format.ISODateTimeFormat; import java.io.File; import java.io.IOException; import java.io.OutputStream; +import java.net.URI; +import java.util.Map; /** */ @@ -62,8 +64,11 @@ public HdfsDataSegmentPusher( this.config = config; this.hadoopConfig = hadoopConfig; this.jsonMapper = jsonMapper; - this.fullyQualifiedStorageDirectory = FileSystem.newInstance(hadoopConfig).makeQualified(new Path(config.getStorageDirectory())) - .toUri().toString(); + Path storageDir = new Path(config.getStorageDirectory()); + this.fullyQualifiedStorageDirectory = FileSystem.newInstance(storageDir.toUri(), hadoopConfig) + .makeQualified(storageDir) + .toUri() + .toString(); log.info("Configured HDFS as deep storage"); } @@ -84,7 +89,7 @@ public String getPathForHadoop() @Override public DataSegment push(File inDir, DataSegment segment) throws IOException { - final String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment); + final String storageDir = this.getStorageDir(segment); log.info( "Copying segment[%s] to HDFS at location[%s/%s]", @@ -115,7 +120,6 @@ public DataSegment push(File inDir, DataSegment segment) throws IOException storageDir, segment.getShardSpec().getPartitionNum() )); - final Path outDescriptorFile = new Path(String.format( "%s/%s/%d_descriptor.json", fullyQualifiedStorageDirectory, @@ -123,7 +127,7 @@ public DataSegment push(File inDir, DataSegment segment) throws IOException segment.getShardSpec().getPartitionNum() )); - dataSegment = segment.withLoadSpec(makeLoadSpec(outIndexFile)) + dataSegment = segment.withLoadSpec(makeLoadSpec(outIndexFile.toUri())) .withSize(size) .withBinaryVersion(SegmentUtils.getVersionFromDir(inDir)); @@ -176,11 +180,6 @@ private void copyFilesWithChecks(final FileSystem fs, final Path from, final Pat } } - private ImmutableMap makeLoadSpec(Path outFile) - { - return ImmutableMap.of("type", "hdfs", "path", outFile.toUri().toString()); - } - private static class HdfsOutputStreamSupplier extends ByteSink { private final FileSystem fs; @@ -198,4 +197,40 @@ public OutputStream openStream() throws IOException return fs.create(descriptorFile); } } + + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + return ImmutableMap.of("type", "hdfs", "path", finalIndexZipFilePath.toString()); + } + + /** + * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in + * path names. So we format paths differently for HDFS. + */ + + @Override + public String getStorageDir(DataSegment segment) + { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), + segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) + ), + segment.getVersion().replaceAll(":", "_") + ); + } + + @Override + public String makeIndexPathName(DataSegment dataSegment, String indexName) + { + return String.format( + "./%s/%d_%s", + this.getStorageDir(dataSegment), + dataSegment.getShardSpec().getPartitionNum(), + indexName + ); + } } diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java index be0ed7d9ae4c..04d9201959d7 100644 --- a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java +++ b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java @@ -20,7 +20,6 @@ package io.druid.segment.loading; import com.google.common.io.ByteStreams; - import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.StringUtils; import io.druid.storage.hdfs.HdfsDataSegmentPuller; @@ -165,12 +164,10 @@ public void testGZ() throws IOException, SegmentLoadingException final URI uri = URI.create(uriBase.toString() + zipPath.toString()); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath)) { - try (final OutputStream gzStream = new GZIPOutputStream(outputStream)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, gzStream); - } - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath); + final OutputStream gzStream = new GZIPOutputStream(outputStream); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, gzStream); } try { Assert.assertFalse(outFile.exists()); @@ -201,10 +198,9 @@ public void testDir() throws IOException, SegmentLoadingException final URI uri = URI.create(uriBase.toString() + perTestPath.toString()); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } try { Assert.assertFalse(outFile.exists()); diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java index 7c8869d9eb84..f5b404a170f2 100644 --- a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java +++ b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java @@ -20,7 +20,6 @@ package io.druid.segment.loading; import com.google.common.io.ByteStreams; - import io.druid.java.util.common.StringUtils; import io.druid.storage.hdfs.HdfsFileTimestampVersionFinder; import org.apache.commons.io.FileUtils; @@ -110,20 +109,18 @@ public void testSimpleLatestVersion() throws IOException, InterruptedException { final Path oldPath = new Path(perTestPath, "555test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Thread.sleep(10); final Path newPath = new Path(perTestPath, "666test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Assert.assertEquals(newPath.toString(), finder.getLatestVersion(oldPath.toUri(), Pattern.compile(".*")).getPath()); @@ -134,20 +131,18 @@ public void testAlreadyLatestVersion() throws IOException, InterruptedException { final Path oldPath = new Path(perTestPath, "555test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Thread.sleep(10); final Path newPath = new Path(perTestPath, "666test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Assert.assertEquals(newPath.toString(), finder.getLatestVersion(newPath.toUri(), Pattern.compile(".*")).getPath()); @@ -166,20 +161,18 @@ public void testSimpleLatestVersionInDir() throws IOException, InterruptedExcept { final Path oldPath = new Path(perTestPath, "555test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Thread.sleep(10); final Path newPath = new Path(perTestPath, "666test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Assert.assertEquals( @@ -193,20 +186,18 @@ public void testSkipMismatch() throws IOException, InterruptedException { final Path oldPath = new Path(perTestPath, "555test.txt"); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Thread.sleep(10); final Path newPath = new Path(perTestPath, "666test.txt2"); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); - try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { - try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { - ByteStreams.copy(inputStream, outputStream); - } + try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath); + final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { + ByteStreams.copy(inputStream, outputStream); } Assert.assertEquals( diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java index f341cd4d65dd..8191e60df2fc 100644 --- a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java +++ b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; @@ -30,20 +31,33 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; +import io.druid.indexer.Bucket; +import io.druid.indexer.HadoopDruidIndexerConfig; +import io.druid.indexer.HadoopIngestionSpec; +import io.druid.indexer.JobHelper; import io.druid.jackson.DefaultObjectMapper; -import io.druid.segment.loading.DataSegmentPusherUtil; +import io.druid.jackson.GranularityModule; +import io.druid.segment.loading.LocalDataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; +import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -51,18 +65,33 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; /** */ public class HdfsDataSegmentPusherTest { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @Rule public final ExpectedException expectedException = ExpectedException.none(); - TestObjectMapper objectMapper = new TestObjectMapper(); + static TestObjectMapper objectMapper = new TestObjectMapper(); + + private HdfsDataSegmentPusher hdfsDataSegmentPusher; + @Before + public void setUp() throws IOException + { + HdfsDataSegmentPusherConfig hdfsDataSegmentPusherConf = new HdfsDataSegmentPusherConfig(); + hdfsDataSegmentPusherConf.setStorageDirectory("path/to/"); + hdfsDataSegmentPusher = new HdfsDataSegmentPusher(hdfsDataSegmentPusherConf, new Configuration(true), objectMapper); + } + static { + objectMapper = new TestObjectMapper(); + objectMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, objectMapper)); + } @Test public void testPushWithScheme() throws Exception @@ -73,8 +102,8 @@ public void testPushWithScheme() throws Exception @Test public void testPushWithBadScheme() throws Exception { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Wrong FS"); + expectedException.expect(IOException.class); + expectedException.expectMessage("No FileSystem for scheme"); testUsingScheme("xyzzy"); // Not reached @@ -133,7 +162,7 @@ private void testUsingScheme(final String scheme) throws Exception String indexUri = String.format( "%s/%s/%d_index.zip", FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(), - DataSegmentPusherUtil.getHdfsStorageDir(segmentToPush), + pusher.getStorageDir(segmentToPush), segmentToPush.getShardSpec().getPartitionNum() ); @@ -146,7 +175,7 @@ private void testUsingScheme(final String scheme) throws Exception indexUri ), segment.getLoadSpec()); // rename directory after push - final String segmentPath = DataSegmentPusherUtil.getHdfsStorageDir(segment); + final String segmentPath = pusher.getStorageDir(segment); File indexFile = new File(String.format( "%s/%s/%d_index.zip", @@ -217,7 +246,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n String indexUri = String.format( "%s/%s/%d_index.zip", FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(), - DataSegmentPusherUtil.getHdfsStorageDir(segments[i]), + pusher.getStorageDir(segments[i]), segments[i].getShardSpec().getPartitionNum() ); @@ -230,7 +259,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n indexUri ), pushedSegment.getLoadSpec()); // rename directory after push - String segmentPath = DataSegmentPusherUtil.getHdfsStorageDir(pushedSegment); + String segmentPath = pusher.getStorageDir(pushedSegment); File indexFile = new File(String.format( "%s/%s/%d_index.zip", @@ -259,7 +288,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n indexUri ), fromDescriptorFileDataSegment.getLoadSpec()); // rename directory after push - segmentPath = DataSegmentPusherUtil.getHdfsStorageDir(fromDescriptorFileDataSegment); + segmentPath = pusher.getStorageDir(fromDescriptorFileDataSegment); indexFile = new File(String.format( "%s/%s/%d_index.zip", @@ -282,7 +311,7 @@ private void testUsingSchemeForMultipleSegments(final String scheme, final int n } } - public class TestObjectMapper extends ObjectMapper + public static class TestObjectMapper extends ObjectMapper { public TestObjectMapper() { @@ -292,10 +321,12 @@ public TestObjectMapper() configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); configure(MapperFeature.AUTO_DETECT_SETTERS, false); configure(SerializationFeature.INDENT_OUTPUT, false); + configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); registerModule(new TestModule().registerSubtypes(new NamedType(NumberedShardSpec.class, "NumberedShardSpec"))); + registerModule(new GranularityModule()); } - public class TestModule extends SimpleModule + public static class TestModule extends SimpleModule { TestModule() { @@ -317,4 +348,250 @@ public Interval deserialize( } } + @Test + public void shouldNotHaveColonsInHdfsStorageDir() throws Exception + { + + Interval interval = new Interval("2011-10-01/2011-10-02"); + ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); + + DataSegment segment = new DataSegment( + "something", + interval, + "brand:new:version", + loadSpec, + Arrays.asList("dim1", "dim2"), + Arrays.asList("met1", "met2"), + NoneShardSpec.instance(), + null, + 1 + ); + + String storageDir = hdfsDataSegmentPusher.getStorageDir(segment); + Assert.assertEquals("something/20111001T000000.000Z_20111002T000000.000Z/brand_new_version", storageDir); + + } + + + @Test + public void shouldMakeHDFSCompliantSegmentOutputPath() + { + HadoopIngestionSpec schema; + + try { + schema = objectMapper.readValue( + "{\n" + + " \"dataSchema\": {\n" + + " \"dataSource\": \"source\",\n" + + " \"metricsSpec\": [],\n" + + " \"granularitySpec\": {\n" + + " \"type\": \"uniform\",\n" + + " \"segmentGranularity\": \"hour\",\n" + + " \"intervals\": [\"2012-07-10/P1D\"]\n" + + " }\n" + + " },\n" + + " \"ioConfig\": {\n" + + " \"type\": \"hadoop\",\n" + + " \"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\"\n" + + " }\n" + + "}", + HadoopIngestionSpec.class + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + //DataSchema dataSchema = new DataSchema("dataSource", null, null, Gra) + //schema = new HadoopIngestionSpec(dataSchema, ioConfig, HadoopTuningConfig.makeDefaultTuningConfig()); + HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( + schema.withTuningConfig( + schema.getTuningConfig() + .withVersion( + "some:brand:new:version" + ) + ) + ); + + Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); + Path path = JobHelper.makeFileNamePath( + new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), + new DistributedFileSystem(), + new DataSegment( + cfg.getSchema().getDataSchema().getDataSource(), + cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), + cfg.getSchema().getTuningConfig().getVersion(), + null, + null, + null, + new NumberedShardSpec(bucket.partitionNum, 5000), + -1, + -1 + ), + JobHelper.INDEX_ZIP, + hdfsDataSegmentPusher + ); + Assert.assertEquals( + "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version" + + "/4712_index.zip", + path.toString() + ); + + path = JobHelper.makeFileNamePath( + new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), + new DistributedFileSystem(), + new DataSegment( + cfg.getSchema().getDataSchema().getDataSource(), + cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), + cfg.getSchema().getTuningConfig().getVersion(), + null, + null, + null, + new NumberedShardSpec(bucket.partitionNum, 5000), + -1, + -1 + ), + JobHelper.DESCRIPTOR_JSON, + hdfsDataSegmentPusher + ); + Assert.assertEquals( + "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version" + + "/4712_descriptor.json", + path.toString() + ); + + path = JobHelper.makeTmpPath( + new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), + new DistributedFileSystem(), + new DataSegment( + cfg.getSchema().getDataSchema().getDataSource(), + cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), + cfg.getSchema().getTuningConfig().getVersion(), + null, + null, + null, + new NumberedShardSpec(bucket.partitionNum, 5000), + -1, + -1 + ), + new TaskAttemptID("abc", 123, TaskType.REDUCE, 1, 0), + hdfsDataSegmentPusher + ); + Assert.assertEquals( + "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version" + + "/4712_index.zip.0", + path.toString() + ); + + } + + @Test + public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() + { + final HadoopIngestionSpec schema; + + try { + schema = objectMapper.readValue( + "{\n" + + " \"dataSchema\": {\n" + + " \"dataSource\": \"the:data:source\",\n" + + " \"metricsSpec\": [],\n" + + " \"granularitySpec\": {\n" + + " \"type\": \"uniform\",\n" + + " \"segmentGranularity\": \"hour\",\n" + + " \"intervals\": [\"2012-07-10/P1D\"]\n" + + " }\n" + + " },\n" + + " \"ioConfig\": {\n" + + " \"type\": \"hadoop\",\n" + + " \"segmentOutputPath\": \"/tmp/dru:id/data:test\"\n" + + " }\n" + + "}", + HadoopIngestionSpec.class + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( + schema.withTuningConfig( + schema.getTuningConfig() + .withVersion( + "some:brand:new:version" + ) + ) + ); + + Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); + Path path = JobHelper.makeFileNamePath( + new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), + new LocalFileSystem(), + new DataSegment( + cfg.getSchema().getDataSchema().getDataSource(), + cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), + cfg.getSchema().getTuningConfig().getVersion(), + null, + null, + null, + new NumberedShardSpec(bucket.partitionNum, 5000), + -1, + -1 + ), + JobHelper.INDEX_ZIP, + new LocalDataSegmentPusher( new LocalDataSegmentPusherConfig(), objectMapper) + ); + Assert.assertEquals( + "file:/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:" + + "version/4712/index.zip", + path.toString() + ); + + path = JobHelper.makeFileNamePath( + new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), + new LocalFileSystem(), + new DataSegment( + cfg.getSchema().getDataSchema().getDataSource(), + cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), + cfg.getSchema().getTuningConfig().getVersion(), + null, + null, + null, + new NumberedShardSpec(bucket.partitionNum, 5000), + -1, + -1 + ), + JobHelper.DESCRIPTOR_JSON, + new LocalDataSegmentPusher( new LocalDataSegmentPusherConfig(), objectMapper) + ); + Assert.assertEquals( + "file:/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:" + + "version/4712/descriptor.json", + path.toString() + ); + + path = JobHelper.makeTmpPath( + new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), + new LocalFileSystem(), + new DataSegment( + cfg.getSchema().getDataSchema().getDataSource(), + cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), + cfg.getSchema().getTuningConfig().getVersion(), + null, + null, + null, + new NumberedShardSpec(bucket.partitionNum, 5000), + -1, + -1 + ), + new TaskAttemptID("abc", 123, TaskType.REDUCE, 1, 0), + new LocalDataSegmentPusher( new LocalDataSegmentPusherConfig(), objectMapper) + ); + Assert.assertEquals( + "file:/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:" + + "version/4712/index.zip.0", + path.toString() + ); + + } } diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java index 86e1a8f0c8d7..4a0cab96d359 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -548,19 +548,19 @@ protected ApproximateHistogram foldMin( } else { Preconditions.checkArgument( mergedPositions.length >= maxSize, - "temp buffer [mergedPositions] too small: length must be at least [%d], got [%d]", + "temp buffer [mergedPositions] too small: length must be at least [%s], got [%s]", maxSize, mergedPositions.length ); Preconditions.checkArgument( mergedBins.length >= maxSize, - "temp buffer [mergedBins] too small: length must be at least [%d], got [%d]", + "temp buffer [mergedBins] too small: length must be at least [%s], got [%s]", maxSize, mergedPositions.length ); Preconditions.checkArgument( deltas.length >= maxSize, - "temp buffer [deltas] too small: length must be at least [%d], got [%d]", + "temp buffer [deltas] too small: length must be at least [%s], got [%s]", maxSize, mergedPositions.length ); diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index ddbf948530b5..948cb8d43511 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -37,6 +37,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -225,7 +226,7 @@ public int getNumBuckets() @Override public List requiredFields() { - return Arrays.asList(fieldName); + return Collections.singletonList(fieldName); } @Override diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java index 9dc37b61b2aa..d1e20092a324 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java @@ -107,6 +107,7 @@ public GenericColumnSerializer getSerializer(IOPeon peon, String column) return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); } + @Override public ObjectStrategy getObjectStrategy() { return new ObjectStrategy() diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index ae52ee52e943..55e1c51a45c7 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -24,7 +24,6 @@ import io.druid.data.input.Row; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; @@ -42,6 +41,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -181,13 +181,13 @@ public void testGroupByWithApproximateHistogramAgg() ) ) .setPostAggregatorSpecs( - Arrays.asList( + Collections.singletonList( new QuantilePostAggregator("quantile", "apphisto", 0.5f) ) ) .build(); - List expectedResults = Arrays.asList( + List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", "marketalias", "upfront", @@ -255,13 +255,13 @@ public void testGroupByWithSameNameComplexPostAgg() ) ) .setPostAggregatorSpecs( - Arrays.asList( + Collections.singletonList( new QuantilePostAggregator("quantile", "quantile", 0.5f) ) ) .build(); - List expectedResults = Arrays.asList( + List expectedResults = Collections.singletonList( GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01T00:00:00.000Z", "marketalias", "upfront", diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index 16ebda20b674..76a814aa3f77 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -47,6 +47,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -54,7 +55,7 @@ @RunWith(Parameterized.class) public class ApproximateHistogramTopNQueryTest { - @Parameterized.Parameters(name="{0}") + @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() throws IOException { return QueryRunnerTestHelper.transformToConstructionFeeder( @@ -62,7 +63,10 @@ public static Iterable constructorFeeder() throws IOException QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( TestQueryRunners.getPool(), - new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ), @@ -79,7 +83,10 @@ public ByteBuffer get() } } ), - new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) @@ -136,109 +143,109 @@ public void testTopNWithApproximateHistogramAgg() ) .build(); - List> expectedResults = Arrays.asList( + List> expectedResults = Collections.singletonList( new Result( new DateTime("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( ImmutableMap.builder() - .put(QueryRunnerTestHelper.marketDimension, "total_market") - .put("rows", 186L) - .put("index", 215679.82879638672D) - .put("addRowsIndexConstant", 215866.82879638672D) - .put(QueryRunnerTestHelper.dependentPostAggMetric, 216053.82879638672D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1743.9217529296875D) - .put("minIndex", 792.3260498046875D) - .put("quantile", 1085.6775f) - .put( - "apphisto", - new Histogram( - new float[]{ - 554.4271240234375f, - 792.3260498046875f, - 1030.2249755859375f, - 1268.1239013671875f, - 1506.0228271484375f, - 1743.9217529296875f - }, - new double[]{ - 0.0D, - 39.42073059082031D, - 103.29110717773438D, - 34.93659591674805D, - 8.351564407348633D - } - ) - ) - .build(), + .put(QueryRunnerTestHelper.marketDimension, "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 216053.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 792.3260498046875D) + .put("quantile", 1085.6775f) + .put( + "apphisto", + new Histogram( + new float[]{ + 554.4271240234375f, + 792.3260498046875f, + 1030.2249755859375f, + 1268.1239013671875f, + 1506.0228271484375f, + 1743.9217529296875f + }, + new double[]{ + 0.0D, + 39.42073059082031D, + 103.29110717773438D, + 34.93659591674805D, + 8.351564407348633D + } + ) + ) + .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.marketDimension, "upfront") - .put("rows", 186L) - .put("index", 192046.1060180664D) - .put("addRowsIndexConstant", 192233.1060180664D) - .put(QueryRunnerTestHelper.dependentPostAggMetric, 192420.1060180664D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1870.06103515625D) - .put("minIndex", 545.9906005859375D) - .put("quantile", 880.9881f) - .put( - "apphisto", - new Histogram( - new float[]{ - 214.97299194335938f, - 545.9906005859375f, - 877.0081787109375f, - 1208.0257568359375f, - 1539.0433349609375f, - 1870.06103515625f - }, - new double[]{ - 0.0D, - 67.53287506103516D, - 72.22068786621094D, - 31.984678268432617D, - 14.261756896972656D - } - ) - ) - .build(), + .put(QueryRunnerTestHelper.marketDimension, "upfront") + .put("rows", 186L) + .put("index", 192046.1060180664D) + .put("addRowsIndexConstant", 192233.1060180664D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 192420.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .put("quantile", 880.9881f) + .put( + "apphisto", + new Histogram( + new float[]{ + 214.97299194335938f, + 545.9906005859375f, + 877.0081787109375f, + 1208.0257568359375f, + 1539.0433349609375f, + 1870.06103515625f + }, + new double[]{ + 0.0D, + 67.53287506103516D, + 72.22068786621094D, + 31.984678268432617D, + 14.261756896972656D + } + ) + ) + .build(), ImmutableMap.builder() - .put(QueryRunnerTestHelper.marketDimension, "spot") - .put("rows", 837L) - .put("index", 95606.57232284546D) - .put("addRowsIndexConstant", 96444.57232284546D) - .put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_9) - .put("maxIndex", 277.2735290527344D) - .put("minIndex", 59.02102279663086D) - .put("quantile", 101.78856f) - .put( - "apphisto", - new Histogram( - new float[]{ - 4.457897186279297f, - 59.02102279663086f, - 113.58415222167969f, - 168.14727783203125f, - 222.7104034423828f, - 277.2735290527344f - }, - new double[]{ - 0.0D, - 462.4309997558594D, - 357.5404968261719D, - 15.022850036621094D, - 2.0056631565093994D - } - ) - ) - .build() + .put(QueryRunnerTestHelper.marketDimension, "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .put("quantile", 101.78856f) + .put( + "apphisto", + new Histogram( + new float[]{ + 4.457897186279297f, + 59.02102279663086f, + 113.58415222167969f, + 168.14727783203125f, + 222.7104034423828f, + 277.2735290527344f + }, + new double[]{ + 0.0D, + 462.4309997558594D, + 357.5404968261719D, + 15.022850036621094D, + 2.0056631565093994D + } + ) + ) + .build() ) ) ) ); - HashMap context = new HashMap(); + HashMap context = new HashMap(); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); } diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index e4eb972d320a..6ccebebaeee6 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -26,6 +26,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryContexts; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -42,6 +43,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.SqlExtractionOperator; import io.druid.sql.calcite.filtration.Filtration; @@ -134,7 +136,7 @@ public void setUp() throws Exception ImmutableSet.of(new QuantileSqlAggregator()), ImmutableSet.of() ); - plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig); + plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig, new ServerConfig()); } @After @@ -200,7 +202,11 @@ public void testQuantileOnFloatAndLongs() throws Exception new QuantilePostAggregator("a6", "a4:agg", 0.999f), new QuantilePostAggregator("a7", "a7:agg", 0.50f) )) - .context(ImmutableMap.of("skipEmptyBuckets", true)) + .context(ImmutableMap.of( + "skipEmptyBuckets", true, + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + )) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); @@ -260,7 +266,11 @@ public void testQuantileOnComplexColumn() throws Exception new QuantilePostAggregator("a5", "a5:agg", 0.999f), new QuantilePostAggregator("a6", "a4:agg", 0.999f) )) - .context(ImmutableMap.of("skipEmptyBuckets", true)) + .context(ImmutableMap.of( + "skipEmptyBuckets", true, + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + )) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); diff --git a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java index 3a87e15b2adc..dd63153ce92c 100644 --- a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java +++ b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java @@ -37,6 +37,7 @@ import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.InvalidMessageException; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.List; @@ -68,7 +69,7 @@ public KafkaEightFirehoseFactory( } @Override - public Firehose connect(final ByteBufferInputRowParser firehoseParser) throws IOException + public Firehose connect(final ByteBufferInputRowParser firehoseParser, File temporaryDirectory) throws IOException { Set newDimExclus = Sets.union( firehoseParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(), diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java index a419efdc6b6d..6b723ff5053d 100644 --- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java +++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java @@ -59,6 +59,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ThreadLocalRandom; /** * @@ -128,6 +129,7 @@ public void close() throws IOException serverProperties.put("zookeeper.connect", zkTestServer.getConnectString() + zkKafkaPath); serverProperties.put("zookeeper.session.timeout.ms", "10000"); serverProperties.put("zookeeper.sync.time.ms", "200"); + serverProperties.put("port", String.valueOf(ThreadLocalRandom.current().nextInt(9999) + 10000)); kafkaConfig = new KafkaConfig(serverProperties); diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index 9ddcd3fc872d..66ca746bd5d9 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -33,6 +33,10 @@ ../../pom.xml + + 0.10.2.0 + + io.druid @@ -55,7 +59,7 @@ org.apache.kafka kafka-clients - 0.10.2.0 + ${apache.kafka.version} @@ -67,7 +71,7 @@ org.apache.kafka kafka_2.11 - 0.10.2.0 + ${apache.kafka.version} test diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java index 6307d77ed12b..f94b18103a1e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java @@ -32,6 +32,7 @@ public class KafkaIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; private static final boolean DEFAULT_PAUSE_AFTER_READ = false; + private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; private final String baseSequenceName; private final KafkaPartitions startPartitions; @@ -40,6 +41,7 @@ public class KafkaIOConfig implements IOConfig private final boolean useTransaction; private final boolean pauseAfterRead; private final Optional minimumMessageTime; + private final boolean skipOffsetGaps; @JsonCreator public KafkaIOConfig( @@ -49,7 +51,8 @@ public KafkaIOConfig( @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("pauseAfterRead") Boolean pauseAfterRead, - @JsonProperty("minimumMessageTime") DateTime minimumMessageTime + @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, + @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps ) { this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName"); @@ -59,6 +62,7 @@ public KafkaIOConfig( this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); + this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; Preconditions.checkArgument( startPartitions.getTopic().equals(endPartitions.getTopic()), @@ -74,7 +78,7 @@ public KafkaIOConfig( Preconditions.checkArgument( endPartitions.getPartitionOffsetMap().get(partition) >= startPartitions.getPartitionOffsetMap() .get(partition), - "end offset must be >= start offset for partition[%d]", + "end offset must be >= start offset for partition[%s]", partition ); } @@ -122,6 +126,12 @@ public Optional getMinimumMessageTime() return minimumMessageTime; } + @JsonProperty + public boolean isSkipOffsetGaps() + { + return skipOffsetGaps; + } + @Override public String toString() { @@ -133,6 +143,7 @@ public String toString() ", useTransaction=" + useTransaction + ", pauseAfterRead=" + pauseAfterRead + ", minimumMessageTime=" + minimumMessageTime + + ", skipOffsetGaps=" + skipOffsetGaps + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 78b878456bb9..37ca3873af06 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -57,6 +57,7 @@ import io.druid.query.DruidMetrics; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; @@ -64,9 +65,9 @@ import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.AppenderatorDriver; +import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import io.druid.segment.realtime.appenderator.Appenderators; -import io.druid.segment.realtime.appenderator.FiniteAppenderatorDriver; -import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import io.druid.segment.realtime.firehose.ChatHandler; @@ -89,7 +90,6 @@ import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; @@ -121,7 +121,6 @@ public enum Status private static final String TYPE = "index_kafka"; private static final Random RANDOM = new Random(); private static final long POLL_TIMEOUT = 100; - private static final long POLL_RETRY_MS = 30000; private static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; @@ -182,6 +181,9 @@ public enum Status private volatile boolean pauseRequested = false; private volatile long pauseMillis = 0; + // This value can be tuned in some tests + private long pollRetryMs = 30000; + @JsonCreator public KafkaIndexTask( @JsonProperty("id") String id, @@ -210,6 +212,12 @@ public KafkaIndexTask( this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); } + @VisibleForTesting + void setPollRetryMs(long retryMs) + { + this.pollRetryMs = retryMs; + } + private static String makeTaskId(String dataSource, int randomBits) { final StringBuilder suffix = new StringBuilder(8); @@ -282,9 +290,11 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception try ( final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); - final FiniteAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); + final AppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); final KafkaConsumer consumer = newConsumer() ) { + toolbox.getDataSegmentServerAnnouncer().announce(); + appenderator = appenderator0; final String topic = ioConfig.getStartPartitions().getTopic(); @@ -355,39 +365,6 @@ public void run() } }; - final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() - { - @Override - public boolean publishSegments(Set segments, Object commitMetadata) throws IOException - { - final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( - ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), - KafkaPartitions.class - ); - - // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { - throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); - } - - final SegmentTransactionalInsertAction action; - - if (ioConfig.isUseTransaction()) { - action = new SegmentTransactionalInsertAction( - segments, - new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), - new KafkaDataSourceMetadata(finalPartitions) - ); - } else { - action = new SegmentTransactionalInsertAction(segments, null, null); - } - - log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); - - return toolbox.getTaskActionClient().submit(action).isSuccess(); - } - }; - Set assignment = assignPartitionsAndSeekToNext(consumer, topic); // Main loop. @@ -437,12 +414,21 @@ public boolean publishSegments(Set segments, Object commitMetadata) if (record.offset() < endOffsets.get(record.partition())) { if (record.offset() != nextOffsets.get(record.partition())) { - throw new ISE( - "WTF?! Got offset[%,d] after offset[%,d] in partition[%d].", - record.offset(), - nextOffsets.get(record.partition()), - record.partition() - ); + if (ioConfig.isSkipOffsetGaps()) { + log.warn( + "Skipped to offset[%,d] after offset[%,d] in partition[%d].", + record.offset(), + nextOffsets.get(record.partition()), + record.partition() + ); + } else { + throw new ISE( + "WTF?! Got offset[%,d] after offset[%,d] in partition[%d].", + record.offset(), + nextOffsets.get(record.partition()), + record.partition() + ); + } } try { @@ -456,15 +442,20 @@ public boolean publishSegments(Set segments, Object commitMetadata) if (!ioConfig.getMinimumMessageTime().isPresent() || !ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp())) { - final SegmentIdentifier identifier = driver.add( + final String sequenceName = sequenceNames.get(record.partition()); + final AppenderatorDriverAddResult addResult = driver.add( row, - sequenceNames.get(record.partition()), - committerSupplier, - publisher, - false - ).lhs; + sequenceName, + committerSupplier + ); - if (identifier == null) { + if (addResult.isOk()) { + // If the number of rows in the segment exceeds the threshold after adding a row, + // move the segment out from the active segments of AppenderatorDriver to make a new segment. + if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { + driver.moveSegmentOut(sequenceName, ImmutableList.of(addResult.getSegmentIdentifier())); + } + } else { // Failure to allocate segment puts determinism at risk, bail out to be safe. // May want configurable behavior here at some point. // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. @@ -515,15 +506,59 @@ public boolean publishSegments(Set segments, Object commitMetadata) status = Status.PUBLISHING; } - final SegmentsAndMetadata published = driver.publishAndWaitHandoff(publisher, committerSupplier.get()); - if (published == null) { + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), + KafkaPartitions.class + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { + throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); + } + + final SegmentTransactionalInsertAction action; + + if (ioConfig.isUseTransaction()) { + action = new SegmentTransactionalInsertAction( + segments, + new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), + new KafkaDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); + + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + + // Supervised kafka tasks are killed by KafkaSupervisor if they are stuck during publishing segments or waiting + // for hand off. See KafkaSupervisorIOConfig.completionTimeout. + final SegmentsAndMetadata published = driver.publish( + publisher, + committerSupplier.get(), + sequenceNames.values() + ).get(); + + final SegmentsAndMetadata handedOff; + if (tuningConfig.getHandoffConditionTimeout() == 0) { + handedOff = driver.registerHandoff(published) + .get(); + } else { + handedOff = driver.registerHandoff(published) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + + if (handedOff == null) { throw new ISE("Transaction failure publishing segments, aborting"); } else { log.info( "Published segments[%s] with metadata[%s].", Joiner.on(", ").join( Iterables.transform( - published.getSegments(), + handedOff.getSegments(), new Function() { @Override @@ -534,7 +569,7 @@ public String apply(DataSegment input) } ) ), - published.getCommitMetadata() + handedOff.getCommitMetadata() ); } } @@ -559,6 +594,8 @@ public String apply(DataSegment input) } } + toolbox.getDataSegmentServerAnnouncer().unannounce(); + return success(); } @@ -628,9 +665,9 @@ public QueryRunner getQueryRunner(Query query) return new QueryRunner() { @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - return query.run(appenderator, responseContext); + return queryPlus.run(appenderator, responseContext); } }; } @@ -826,7 +863,7 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox ioConfig.getStartPartitions().getPartitionOffsetMap().size()); return Appenderators.createRealtime( dataSchema, - tuningConfig.withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")) + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()) .withMaxRowsInMemory(maxRowsInMemoryPerPartition), metrics, toolbox.getSegmentPusher(), @@ -842,21 +879,18 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox ); } - private FiniteAppenderatorDriver newDriver( + private AppenderatorDriver newDriver( final Appenderator appenderator, final TaskToolbox toolbox, final FireDepartmentMetrics metrics ) { - return new FiniteAppenderatorDriver( + return new AppenderatorDriver( appenderator, new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), - tuningConfig.getMaxRowsPerSegment(), - Long.MAX_VALUE, // KafkaIndexTask doesn't support this parameter yet - tuningConfig.getHandoffConditionTimeout(), metrics ); } @@ -1029,10 +1063,10 @@ private void possiblyResetOffsetsOrWait( if (doReset) { sendResetRequestAndWait(resetPartitions, taskToolbox); } else { - log.warn("Retrying in %dms", POLL_RETRY_MS); + log.warn("Retrying in %dms", pollRetryMs); pollRetryLock.lockInterruptibly(); try { - long nanos = TimeUnit.MILLISECONDS.toNanos(POLL_RETRY_MS); + long nanos = TimeUnit.MILLISECONDS.toNanos(pollRetryMs); while (nanos > 0L && !pauseRequested && !stopRequested) { nanos = isAwaitingRetry.awaitNanos(nanos); } @@ -1066,10 +1100,10 @@ private void sendResetRequestAndWait(Map outOfRangePartiti .emit(); // wait for being killed by supervisor try { - Thread.sleep(Long.MAX_VALUE); + pause(-1); } catch (InterruptedException e) { - throw new RuntimeException("Got interrupted while waiting to be killed"); + throw new RuntimeException("Got interrupted while pausing task"); } } else { log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index 71f5ea60f182..b2294bba0947 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -60,7 +60,7 @@ public class KafkaIndexTaskClient { - public class NoTaskLocationException extends RuntimeException + public static class NoTaskLocationException extends RuntimeException { public NoTaskLocationException(String message) { @@ -68,7 +68,7 @@ public NoTaskLocationException(String message) } } - public class TaskNotRunnableException extends RuntimeException + public static class TaskNotRunnableException extends RuntimeException { public TaskNotRunnableException(String message) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 080ab3cf876c..d257d7c98ec5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -42,6 +42,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private final IndexSpec indexSpec; private final boolean buildV9Directly; private final boolean reportParseExceptions; + @Deprecated private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; @@ -98,43 +99,41 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) ); } + @Override @JsonProperty public int getMaxRowsInMemory() { return maxRowsInMemory; } - @Override - public long getMaxPersistedSegmentsBytes() - { - // This option is not supported yet - throw new UnsupportedOperationException(); - } - @JsonProperty public int getMaxRowsPerSegment() { return maxRowsPerSegment; } + @Override @JsonProperty public Period getIntermediatePersistPeriod() { return intermediatePersistPeriod; } + @Override @JsonProperty public File getBasePersistDirectory() { return basePersistDirectory; } + @Override @JsonProperty public int getMaxPendingPersists() { return maxPendingPersists; } + @Override @JsonProperty public IndexSpec getIndexSpec() { @@ -147,12 +146,14 @@ public boolean getBuildV9Directly() return buildV9Directly; } + @Override @JsonProperty public boolean isReportParseExceptions() { return reportParseExceptions; } + @Deprecated @JsonProperty public long getHandoffConditionTimeout() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index a3a55e3f792d..3634ece1faf7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -40,6 +40,8 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.concurrent.Execs; import io.druid.indexing.common.TaskInfoProvider; import io.druid.indexing.common.TaskLocation; @@ -67,6 +69,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.collect.JavaCompatUtils; import io.druid.metadata.EntryExistsException; +import io.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.commons.codec.digest.DigestUtils; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; @@ -76,8 +79,8 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.HashMap; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -93,6 +96,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.BiFunction; /** * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a @@ -121,7 +125,7 @@ public class KafkaSupervisor implements Supervisor * time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [taskGroups] * map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]). */ - private class TaskGroup + private static class TaskGroup { // This specifies the partitions and starting offsets for this task group. It is set on group creation from the data // in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in @@ -146,7 +150,7 @@ Set taskIds() } } - private class TaskData + private static class TaskData { TaskStatus status; DateTime startTime; @@ -180,6 +184,8 @@ private class TaskData private final KafkaIndexTaskClient taskClient; private final ObjectMapper sortingMapper; private final KafkaSupervisorSpec spec; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final String dataSource; private final KafkaSupervisorIOConfig ioConfig; private final KafkaSupervisorTuningConfig tuningConfig; @@ -200,9 +206,14 @@ private class TaskData private volatile DateTime firstRunTime; private volatile KafkaConsumer consumer; + private volatile KafkaConsumer lagComputingConsumer; private volatile boolean started = false; private volatile boolean stopped = false; + private final ScheduledExecutorService metricEmittingExec; + // used while reporting lag + private final Map lastCurrentOffsets = new HashMap<>(); + public KafkaSupervisor( final TaskStorage taskStorage, final TaskMaster taskMaster, @@ -217,6 +228,8 @@ public KafkaSupervisor( this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); this.spec = spec; + this.emitter = spec.getEmitter(); + this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -225,6 +238,7 @@ public KafkaSupervisor( this.supervisorId = String.format("KafkaSupervisor-%s", dataSource); this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); + this.metricEmittingExec = Execs.scheduledSingleThreaded(supervisorId + "-Emitter-%d"); int workerThreads = (this.tuningConfig.getWorkerThreads() != null ? this.tuningConfig.getWorkerThreads() @@ -302,6 +316,7 @@ public void start() try { consumer = getKafkaConsumer(); + lagComputingConsumer = getKafkaConsumer(); exec.submit( new Runnable() @@ -337,6 +352,13 @@ public void run() TimeUnit.MILLISECONDS ); + metricEmittingExec.scheduleAtFixedRate( + computeAndEmitLag(taskClient), + ioConfig.getStartDelay().getMillis() + 10000, // wait for tasks to start up + Math.max(monitorSchedulerConfig.getEmitterPeriod().getMillis(), 60 * 1000), + TimeUnit.MILLISECONDS + ); + started = true; log.info( "Started KafkaSupervisor[%s], first run in [%s], with spec: [%s]", @@ -349,6 +371,9 @@ public void run() if (consumer != null) { consumer.close(); } + if (lagComputingConsumer != null) { + lagComputingConsumer.close(); + } log.makeAlert(e, "Exception starting KafkaSupervisor[%s]", dataSource) .emit(); throw Throwables.propagate(e); @@ -366,6 +391,7 @@ public void stop(boolean stopGracefully) try { scheduledExec.shutdownNow(); // stop recurring executions + metricEmittingExec.shutdownNow(); Optional taskRunner = taskMaster.getTaskRunner(); if (taskRunner.isPresent()) { @@ -499,6 +525,7 @@ private class ShutdownNotice implements Notice public void handle() throws InterruptedException, ExecutionException, TimeoutException { consumer.close(); + lagComputingConsumer.close(); synchronized (stopLock) { stopped = true; @@ -1374,7 +1401,8 @@ private void createKafkaTasksForGroup(int groupId, int replicas) consumerProperties, true, false, - minimumMessageTime + minimumMessageTime, + ioConfig.isSkipOffsetGaps() ); for (int i = 0; i < replicas; i++) { @@ -1677,4 +1705,103 @@ public void run() } }; } + + private Runnable computeAndEmitLag(final KafkaIndexTaskClient taskClient) + { + return new Runnable() + { + @Override + public void run() + { + try { + final Map> topics = lagComputingConsumer.listTopics(); + final List partitionInfoList = topics.get(ioConfig.getTopic()); + lagComputingConsumer.assign( + Lists.transform(partitionInfoList, new Function() + { + @Override + public TopicPartition apply(PartitionInfo input) + { + return new TopicPartition(ioConfig.getTopic(), input.partition()); + } + }) + ); + final Map offsetsResponse = new ConcurrentHashMap<>(); + final List> futures = Lists.newArrayList(); + for (TaskGroup taskGroup : taskGroups.values()) { + for (String taskId : taskGroup.taskIds()) { + futures.add(Futures.transform( + taskClient.getCurrentOffsetsAsync(taskId, false), + new Function, Void>() + { + @Override + public Void apply(Map taskResponse) + { + if (taskResponse != null) { + for (final Map.Entry partitionOffsets : taskResponse.entrySet()) { + offsetsResponse.compute(partitionOffsets.getKey(), new BiFunction() + { + @Override + public Long apply(Integer key, Long existingOffsetInMap) + { + // If existing value is null use the offset returned by task + // otherwise use the max (makes sure max offset is taken from replicas) + return existingOffsetInMap == null + ? partitionOffsets.getValue() + : Math.max(partitionOffsets.getValue(), existingOffsetInMap); + } + }); + } + } + return null; + } + } + ) + ); + } + } + // not using futureTimeoutInSeconds as its min value is 120 seconds + // and minimum emission period for this metric is 60 seconds + Futures.successfulAsList(futures).get(30, TimeUnit.SECONDS); + + // for each partition, seek to end to get the highest offset + // check the offsetsResponse map for the latest consumed offset + // if partition info not present in offsetsResponse then use lastCurrentOffsets map + // if not present there as well, fail the compute + + long lag = 0; + for (PartitionInfo partitionInfo : partitionInfoList) { + long diff; + final TopicPartition topicPartition = new TopicPartition(ioConfig.getTopic(), partitionInfo.partition()); + lagComputingConsumer.seekToEnd(ImmutableList.of(topicPartition)); + if (offsetsResponse.get(topicPartition.partition()) != null) { + diff = lagComputingConsumer.position(topicPartition) - offsetsResponse.get(topicPartition.partition()); + lastCurrentOffsets.put(topicPartition.partition(), offsetsResponse.get(topicPartition.partition())); + } else if (lastCurrentOffsets.get(topicPartition.partition()) != null) { + diff = lagComputingConsumer.position(topicPartition) - lastCurrentOffsets.get(topicPartition.partition()); + } else { + throw new ISE("Could not find latest consumed offset for partition [%d]", topicPartition.partition()); + } + lag += diff; + log.debug( + "Topic - [%s] Partition - [%d] : Partition lag [%,d], Total lag so far [%,d]", + topicPartition.topic(), + topicPartition.partition(), + diff, + lag + ); + } + emitter.emit( + ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kafka/lag", lag) + ); + } + catch (InterruptedException e) { + // do nothing, probably we are shutting down + } + catch (Exception e) { + log.warn(e, "Unable to compute Kafka lag"); + } + } + }; + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 516d9493c1e1..395531b59c31 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -39,9 +39,10 @@ public class KafkaSupervisorIOConfig private final Map consumerProperties; private final Duration startDelay; private final Duration period; - private final Boolean useEarliestOffset; + private final boolean useEarliestOffset; private final Duration completionTimeout; private final Optional lateMessageRejectionPeriod; + private final boolean skipOffsetGaps; @JsonCreator public KafkaSupervisorIOConfig( @@ -54,7 +55,8 @@ public KafkaSupervisorIOConfig( @JsonProperty("period") Period period, @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, @JsonProperty("completionTimeout") Period completionTimeout, - @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod + @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, + @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps ) { this.topic = Preconditions.checkNotNull(topic, "topic"); @@ -64,16 +66,17 @@ public KafkaSupervisorIOConfig( String.format("consumerProperties must contain entry for [%s]", BOOTSTRAP_SERVERS_KEY) ); - this.replicas = (replicas != null ? replicas : 1); - this.taskCount = (taskCount != null ? taskCount : 1); + this.replicas = replicas != null ? replicas : 1; + this.taskCount = taskCount != null ? taskCount : 1; this.taskDuration = defaultDuration(taskDuration, "PT1H"); this.startDelay = defaultDuration(startDelay, "PT5S"); this.period = defaultDuration(period, "PT30S"); - this.useEarliestOffset = (useEarliestOffset != null ? useEarliestOffset : false); + this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false; this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); - this.lateMessageRejectionPeriod = (lateMessageRejectionPeriod == null - ? Optional.absent() - : Optional.of(lateMessageRejectionPeriod.toStandardDuration())); + this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null + ? Optional.absent() + : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); + this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : false; } @JsonProperty @@ -119,7 +122,7 @@ public Duration getPeriod() } @JsonProperty - public Boolean isUseEarliestOffset() + public boolean isUseEarliestOffset() { return useEarliestOffset; } @@ -136,6 +139,12 @@ public Optional getLateMessageRejectionPeriod() return lateMessageRejectionPeriod; } + @JsonProperty + public boolean isSkipOffsetGaps() + { + return skipOffsetGaps; + } + @Override public String toString() { @@ -150,6 +159,7 @@ public String toString() ", useEarliestOffset=" + useEarliestOffset + ", completionTimeout=" + completionTimeout + ", lateMessageRejectionPeriod=" + lateMessageRejectionPeriod + + ", skipOffsetGaps=" + skipOffsetGaps + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java index 1ffd8b14e391..a6e0c761c1ae 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorReport.java @@ -31,7 +31,7 @@ public class KafkaSupervisorReport extends SupervisorReport { - public class KafkaSupervisorReportPayload + public static class KafkaSupervisorReportPayload { private final String dataSource; private final String topic; diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index a536f87fe3fd..2bcd5c70ccf6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.metamx.emitter.service.ServiceEmitter; import io.druid.guice.annotations.Json; import io.druid.indexing.kafka.KafkaIndexTaskClientFactory; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -33,6 +34,7 @@ import io.druid.indexing.overlord.supervisor.Supervisor; import io.druid.indexing.overlord.supervisor.SupervisorSpec; import io.druid.segment.indexing.DataSchema; +import io.druid.server.metrics.DruidMonitorSchedulerConfig; import java.util.List; import java.util.Map; @@ -49,6 +51,8 @@ public class KafkaSupervisorSpec implements SupervisorSpec private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final KafkaIndexTaskClientFactory kafkaIndexTaskClientFactory; private final ObjectMapper mapper; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; @JsonCreator public KafkaSupervisorSpec( @@ -60,8 +64,10 @@ public KafkaSupervisorSpec( @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @JacksonInject KafkaIndexTaskClientFactory kafkaIndexTaskClientFactory, - @JacksonInject @Json ObjectMapper mapper - ) + @JacksonInject @Json ObjectMapper mapper, + @JacksonInject ServiceEmitter emitter, + @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig + ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.tuningConfig = tuningConfig != null @@ -91,6 +97,8 @@ public KafkaSupervisorSpec( this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; this.kafkaIndexTaskClientFactory = kafkaIndexTaskClientFactory; this.mapper = mapper; + this.emitter = emitter; + this.monitorSchedulerConfig = monitorSchedulerConfig; } @JsonProperty @@ -117,12 +125,22 @@ public Map getContext() return context; } + public ServiceEmitter getEmitter() + { + return emitter; + } + @Override public String getId() { return dataSchema.getDataSource(); } + public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() + { + return monitorSchedulerConfig; + } + @Override public Supervisor createSupervisor() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 0e28d4abcb23..c04771a6ae69 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -44,7 +44,7 @@ public KafkaSupervisorTuningConfig( @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, // for backward compatibility @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, @JsonProperty("workerThreads") Integer workerThreads, @JsonProperty("chatThreads") Integer chatThreads, @@ -62,6 +62,8 @@ public KafkaSupervisorTuningConfig( indexSpec, buildV9Directly, reportParseExceptions, + // Supervised kafka tasks should respect KafkaSupervisorIOConfig.completionTimeout instead of + // handoffConditionTimeout handoffConditionTimeout, resetOffsetAutomatically ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIOConfigTest.java index de63ab14904e..c18cc9588050 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIOConfigTest.java @@ -74,6 +74,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(true, config.isUseTransaction()); Assert.assertEquals(false, config.isPauseAfterRead()); Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); + Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); } @Test @@ -87,7 +88,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" + " \"useTransaction\": false,\n" + " \"pauseAfterRead\": true,\n" - + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" + + " \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n" + + " \"skipOffsetGaps\": true\n" + "}"; KafkaIOConfig config = (KafkaIOConfig) mapper.readValue( @@ -108,6 +110,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(false, config.isUseTransaction()); Assert.assertEquals(true, config.isPauseAfterRead()); Assert.assertEquals(new DateTime("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); + Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); } @Test diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 2132d0623bd4..e20cc8bfd599 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -112,6 +112,7 @@ import io.druid.segment.loading.StorageLocationConfig; import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.timeline.DataSegment; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; @@ -121,8 +122,10 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -144,66 +147,78 @@ @RunWith(Parameterized.class) public class KafkaIndexTaskTest { + private static final Logger log = new Logger(KafkaIndexTaskTest.class); + private static final ObjectMapper objectMapper = new DefaultObjectMapper(); + private static final long POLL_RETRY_MS = 100; + + private static TestingCluster zkServer; + private static TestBroker kafkaServer; + private static ServiceEmitter emitter; + private static ListeningExecutorService taskExec; + private static int topicPostfix; + + private final List runningTasks = Lists.newArrayList(); private final boolean buildV9Directly; + private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; private boolean doHandoff = true; - private TestingCluster zkServer; - private TestBroker kafkaServer; - private ServiceEmitter emitter; - private ListeningExecutorService taskExec; private TaskToolboxFactory toolboxFactory; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private TaskStorage taskStorage; private TaskLockbox taskLockbox; private File directory; - - private final List runningTasks = Lists.newArrayList(); - - private static final Logger log = new Logger(KafkaIndexTaskTest.class); - private static final ObjectMapper objectMapper = new DefaultObjectMapper(); - - private static final DataSchema DATA_SCHEMA; - - private static final List> RECORDS = ImmutableList.of( - new ProducerRecord("topic0", 0, null, JB("2008", "a", "y", 1.0f)), - new ProducerRecord("topic0", 0, null, JB("2009", "b", "y", 1.0f)), - new ProducerRecord("topic0", 0, null, JB("2010", "c", "y", 1.0f)), - new ProducerRecord("topic0", 0, null, JB("2011", "d", "y", 1.0f)), - new ProducerRecord("topic0", 0, null, JB("2011", "e", "y", 1.0f)), - new ProducerRecord("topic0", 0, null, "unparseable".getBytes()), - new ProducerRecord("topic0", 0, null, null), - new ProducerRecord("topic0", 0, null, JB("2013", "f", "y", 1.0f)), - new ProducerRecord("topic0", 1, null, JB("2012", "g", "y", 1.0f)), - new ProducerRecord("topic0", 1, null, JB("2011", "h", "y", 1.0f)) + private String topic; + private List> records; + + private static final DataSchema DATA_SCHEMA = new DataSchema( + "test_ds", + objectMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2")), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + Charsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + objectMapper ); - static { - DATA_SCHEMA = new DataSchema( - "test_ds", - objectMapper.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2")), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - Charsets.UTF_8.name() - ), - Map.class - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - objectMapper + private static List> generateRecords(String topic) + { + return ImmutableList.of( + new ProducerRecord(topic, 0, null, JB("2008", "a", "y", 1.0f)), + new ProducerRecord(topic, 0, null, JB("2009", "b", "y", 1.0f)), + new ProducerRecord(topic, 0, null, JB("2010", "c", "y", 1.0f)), + new ProducerRecord(topic, 0, null, JB("2011", "d", "y", 1.0f)), + new ProducerRecord(topic, 0, null, JB("2011", "e", "y", 1.0f)), + new ProducerRecord(topic, 0, null, "unparseable".getBytes()), + new ProducerRecord(topic, 0, null, null), + new ProducerRecord(topic, 0, null, JB("2013", "f", "y", 1.0f)), + new ProducerRecord(topic, 1, null, JB("2012", "g", "y", 1.0f)), + new ProducerRecord(topic, 1, null, JB("2011", "h", "y", 1.0f)) ); } + private static String getTopicName() + { + return "topic" + topicPostfix++; + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + @Parameterized.Parameters(name = "buildV9Directly = {0}") public static Iterable constructorFeeder() { @@ -215,14 +230,11 @@ public KafkaIndexTaskTest(boolean buildV9Directly) this.buildV9Directly = buildV9Directly; } - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); - @Before - public void setUp() throws Exception + @BeforeClass + public static void setupClass() throws Exception { emitter = new ServiceEmitter( "service", @@ -236,14 +248,12 @@ public void setUp() throws Exception emitter.start(); EmittingLogger.registerEmitter(emitter); - makeToolboxFactory(); - zkServer = new TestingCluster(1); zkServer.start(); kafkaServer = new TestBroker( zkServer.getConnectString(), - tempFolder.newFolder(), + null, 1, ImmutableMap.of("num.partitions", "2") ); @@ -254,17 +264,22 @@ public void setUp() throws Exception Execs.makeThreadFactory("kafka-task-test-%d") ) ); + } + @Before + public void setupTest() throws IOException + { handoffConditionTimeout = 0; reportParseExceptions = false; doHandoff = true; + topic = getTopicName(); + records = generateRecords(topic); + makeToolboxFactory(); } @After - public void tearDown() throws Exception + public void tearDownTest() { - emitter.close(); - synchronized (runningTasks) { for (Task task : runningTasks) { task.stopGracefully(); @@ -273,6 +288,12 @@ public void tearDown() throws Exception runningTasks.clear(); } + destroyToolboxFactory(); + } + + @AfterClass + public static void tearDownClass() throws Exception + { taskExec.shutdown(); taskExec.awaitTermination(9999, TimeUnit.DAYS); @@ -282,7 +303,7 @@ public void tearDown() throws Exception zkServer.stop(); zkServer = null; - destroyToolboxFactory(); + emitter.close(); } @Test(timeout = 60_000L) @@ -290,7 +311,7 @@ public void testRunAfterDataInserted() throws Exception { // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -299,12 +320,13 @@ public void testRunAfterDataInserted() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -325,7 +347,7 @@ public void testRunAfterDataInserted() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -341,12 +363,13 @@ public void testRunBeforeDataInserted() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -361,7 +384,7 @@ public void testRunBeforeDataInserted() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -379,7 +402,7 @@ public void testRunBeforeDataInserted() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -395,12 +418,13 @@ public void testRunWithMinimumMessageTime() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 0L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - new DateTime("2010") + new DateTime("2010"), + false ), null, null @@ -415,7 +439,7 @@ public void testRunWithMinimumMessageTime() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -433,7 +457,7 @@ public void testRunWithMinimumMessageTime() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -447,7 +471,7 @@ public void testRunOnNothing() throws Exception { // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -456,12 +480,13 @@ public void testRunOnNothing() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -488,7 +513,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -497,12 +522,13 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -523,7 +549,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -540,7 +566,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -549,12 +575,13 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -575,7 +602,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -591,7 +618,7 @@ public void testReportParseExceptions() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -600,12 +627,13 @@ public void testReportParseExceptions() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 7L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -633,12 +661,13 @@ public void testRunReplicas() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -647,12 +676,13 @@ public void testRunReplicas() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -663,7 +693,7 @@ public void testRunReplicas() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -685,7 +715,7 @@ public void testRunReplicas() throws Exception SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -701,12 +731,13 @@ public void testRunConflicting() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -715,12 +746,13 @@ public void testRunConflicting() throws Exception null, new KafkaIOConfig( "sequence1", - new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 8L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 8L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -728,7 +760,7 @@ public void testRunConflicting() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -754,7 +786,7 @@ public void testRunConflicting() throws Exception SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -770,12 +802,13 @@ public void testRunConflictingWithoutTransactions() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), false, false, - null + null, + false ), null, null @@ -784,12 +817,13 @@ public void testRunConflictingWithoutTransactions() throws Exception null, new KafkaIOConfig( "sequence1", - new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 8L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 8L)), kafkaServer.consumerProperties(), false, false, - null + null, + false ), null, null @@ -797,7 +831,7 @@ public void testRunConflictingWithoutTransactions() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -844,12 +878,13 @@ public void testRunOneTaskTwoPartitions() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L, 1, 0L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -859,9 +894,10 @@ public void testRunOneTaskTwoPartitions() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } + kafkaProducer.flush(); } // Wait for tasks to exit @@ -879,7 +915,7 @@ public void testRunOneTaskTwoPartitions() throws Exception SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -901,12 +937,13 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -915,12 +952,13 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, new KafkaIOConfig( "sequence1", - new KafkaPartitions("topic0", ImmutableMap.of(1, 0L)), - new KafkaPartitions("topic0", ImmutableMap.of(1, 1L)), + new KafkaPartitions(topic, ImmutableMap.of(1, 0L)), + new KafkaPartitions(topic, ImmutableMap.of(1, 1L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -931,7 +969,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -954,7 +992,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 1L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 1L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -971,12 +1009,13 @@ public void testRestore() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -986,7 +1025,7 @@ public void testRestore() throws Exception // Insert some data, but not enough for the task to finish try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { + for (ProducerRecord record : Iterables.limit(records, 4)) { kafkaProducer.send(record).get(); } } @@ -1006,12 +1045,13 @@ public void testRestore() throws Exception task1.getId(), new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -1021,7 +1061,7 @@ public void testRestore() throws Exception // Insert remaining data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { + for (ProducerRecord record : Iterables.skip(records, 4)) { kafkaProducer.send(record).get(); } } @@ -1042,7 +1082,7 @@ public void testRestore() throws Exception SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1058,12 +1098,13 @@ public void testRunWithPauseAndResume() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -1073,9 +1114,10 @@ public void testRunWithPauseAndResume() throws Exception // Insert some data, but not enough for the task to finish try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { + for (ProducerRecord record : Iterables.limit(records, 4)) { kafkaProducer.send(record).get(); } + kafkaProducer.flush(); } while (countEvents(task) != 2) { @@ -1095,7 +1137,7 @@ public void testRunWithPauseAndResume() throws Exception // Insert remaining data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { + for (ProducerRecord record : Iterables.skip(records, 4)) { kafkaProducer.send(record).get(); } } @@ -1125,7 +1167,7 @@ public void testRunWithPauseAndResume() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1141,12 +1183,13 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 1L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 1L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), kafkaServer.consumerProperties(), true, true, - null + null, + false ), null, null @@ -1155,7 +1198,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception final ListenableFuture future = runTask(task); try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -1211,7 +1254,7 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception SegmentDescriptor desc3 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 6L))), + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 6L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1228,12 +1271,13 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, null @@ -1257,7 +1301,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva { // Insert data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - for (ProducerRecord record : RECORDS) { + for (ProducerRecord record : records) { kafkaProducer.send(record).get(); } } @@ -1266,12 +1310,13 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva null, new KafkaIOConfig( "sequence0", - new KafkaPartitions("topic0", ImmutableMap.of(0, 200L)), - new KafkaPartitions("topic0", ImmutableMap.of(0, 500L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 200L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 500L)), kafkaServer.consumerProperties(), true, false, - null + null, + false ), null, true @@ -1280,15 +1325,13 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva runTask(task); while (!task.getStatus().equals(KafkaIndexTask.Status.READING)) { - Thread.sleep(2000); + Thread.sleep(20); } - int i = 0; - while(i++ < 5) { + for (int i = 0; i < 5; i++) { Assert.assertEquals(task.getStatus(), KafkaIndexTask.Status.READING); // Offset should not be reset Assert.assertTrue(task.getCurrentOffsets().get(0) == 200L); - Thread.sleep(2000); } } @@ -1361,7 +1404,7 @@ private KafkaIndexTask createTask( handoffConditionTimeout, resetOffsetAutomatically ); - return new KafkaIndexTask( + final KafkaIndexTask task = new KafkaIndexTask( taskId, null, DATA_SCHEMA, @@ -1370,6 +1413,8 @@ private KafkaIndexTask createTask( null, null ); + task.setPollRetryMs(POLL_RETRY_MS); + return task; } private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() @@ -1500,6 +1545,7 @@ public void close() null, // DataSegmentMover null, // DataSegmentArchiver new TestDataSegmentAnnouncer(), + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, makeTimeseriesOnlyConglomerate(), MoreExecutors.sameThreadExecutor(), // queryExecutorService diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index 70d2766ef81e..d42b44c5475d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -104,7 +104,18 @@ public void testSerdeWithNonDefaults() throws Exception @Test public void testCopyOf() throws Exception { - KafkaTuningConfig original = new KafkaTuningConfig(1, 2, new Period("PT3S"), new File("/tmp/xxx"), 4, new IndexSpec(), true, true, 5L, null); + KafkaTuningConfig original = new KafkaTuningConfig( + 1, + 2, + new Period("PT3S"), + new File("/tmp/xxx"), + 4, + new IndexSpec(), + true, + true, + 5L, + null + ); KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original); Assert.assertEquals(1, copy.getMaxRowsInMemory()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java index e94f4a740feb..b453e705cfa3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java @@ -73,6 +73,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(false, config.isUseEarliestOffset()); Assert.assertEquals(Duration.standardMinutes(30), config.getCompletionTimeout()); Assert.assertFalse("lateMessageRejectionPeriod", config.getLateMessageRejectionPeriod().isPresent()); + Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); } @Test @@ -89,7 +90,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"period\": \"PT10S\",\n" + " \"useEarliestOffset\": true,\n" + " \"completionTimeout\": \"PT45M\",\n" - + " \"lateMessageRejectionPeriod\": \"PT1H\"\n" + + " \"lateMessageRejectionPeriod\": \"PT1H\",\n" + + " \"skipOffsetGaps\": true\n" + "}"; KafkaSupervisorIOConfig config = mapper.readValue( @@ -111,6 +113,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(true, config.isUseEarliestOffset()); Assert.assertEquals(Duration.standardMinutes(45), config.getCompletionTimeout()); Assert.assertEquals(Duration.standardHours(1), config.getLateMessageRejectionPeriod().get()); + Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); } @Test diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index d22d1772810d..6d2c8f66dbe8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -65,6 +65,8 @@ import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.FireDepartment; +import io.druid.server.metrics.DruidMonitorSchedulerConfig; +import io.druid.server.metrics.NoopServiceEmitter; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -77,15 +79,16 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; +import org.junit.BeforeClass; import org.junit.Test; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -106,7 +109,7 @@ public class KafkaSupervisorTest extends EasyMockSupport { private static final ObjectMapper objectMapper = new DefaultObjectMapper(); - private static final String KAFKA_TOPIC = "testTopic"; + private static final String TOPIC_PREFIX = "testTopic"; private static final String DATASOURCE = "testDS"; private static final int NUM_PARTITIONS = 3; private static final int TEST_CHAT_THREADS = 3; @@ -114,12 +117,15 @@ public class KafkaSupervisorTest extends EasyMockSupport private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); - private int numThreads; - private TestingCluster zkServer; - private TestBroker kafkaServer; + private static TestingCluster zkServer; + private static TestBroker kafkaServer; + private static String kafkaHost; + private static DataSchema dataSchema; + private static int topicPostfix; + + private final int numThreads; + private KafkaSupervisor supervisor; - private String kafkaHost; - private DataSchema dataSchema; private KafkaSupervisorTuningConfig tuningConfig; private TaskStorage taskStorage; private TaskMaster taskMaster; @@ -127,9 +133,12 @@ public class KafkaSupervisorTest extends EasyMockSupport private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private KafkaIndexTaskClient taskClient; private TaskQueue taskQueue; + private String topic; - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); + private static String getTopic() + { + return TOPIC_PREFIX + topicPostfix++; + } @Parameterized.Parameters(name = "numThreads = {0}") public static Iterable constructorFeeder() @@ -142,22 +151,15 @@ public KafkaSupervisorTest(int numThreads) this.numThreads = numThreads; } - @Before - public void setUp() throws Exception + @BeforeClass + public static void setupClass() throws Exception { - taskStorage = createMock(TaskStorage.class); - taskMaster = createMock(TaskMaster.class); - taskRunner = createMock(TaskRunner.class); - indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); - taskClient = createMock(KafkaIndexTaskClient.class); - taskQueue = createMock(TaskQueue.class); - zkServer = new TestingCluster(1); zkServer.start(); kafkaServer = new TestBroker( zkServer.getConnectString(), - tempFolder.newFolder(), + null, 1, ImmutableMap.of("num.partitions", String.valueOf(NUM_PARTITIONS)) ); @@ -165,6 +167,18 @@ public void setUp() throws Exception kafkaHost = String.format("localhost:%d", kafkaServer.getPort()); dataSchema = getDataSchema(DATASOURCE); + } + + @Before + public void setupTest() throws Exception + { + taskStorage = createMock(TaskStorage.class); + taskMaster = createMock(TaskMaster.class); + taskRunner = createMock(TaskRunner.class); + indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); + taskClient = createMock(KafkaIndexTaskClient.class); + taskQueue = createMock(TaskQueue.class); + tuningConfig = new KafkaSupervisorTuningConfig( 1000, 50000, @@ -182,24 +196,30 @@ public void setUp() throws Exception TEST_HTTP_TIMEOUT, TEST_SHUTDOWN_TIMEOUT ); + + topic = getTopic(); } @After - public void tearDown() throws Exception + public void tearDownTest() throws Exception + { + supervisor = null; + } + + @AfterClass + public static void tearDownClass() throws IOException { kafkaServer.close(); kafkaServer = null; zkServer.stop(); zkServer = null; - - supervisor = null; } @Test public void testNoInitialState() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); addSomeEvents(1); Capture captured = Capture.newInstance(); @@ -230,22 +250,52 @@ public void testNoInitialState() throws Exception Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); + Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); - Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic()); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); - Assert.assertEquals(KAFKA_TOPIC, taskConfig.getEndPartitions().getTopic()); + Assert.assertEquals(topic, taskConfig.getEndPartitions().getTopic()); Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); } + @Test + public void testSkipOffsetGaps() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, true); + addSomeEvents(1); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KafkaIndexTask task = captured.getValue(); + KafkaIOConfig taskConfig = task.getIOConfig(); + + Assert.assertTrue("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); + } + @Test public void testMultiTask() throws Exception { - supervisor = getSupervisor(1, 2, true, "PT1H", null); + supervisor = getSupervisor(1, 2, true, "PT1H", null, false); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -282,7 +332,7 @@ public void testMultiTask() throws Exception @Test public void testReplicas() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null); + supervisor = getSupervisor(2, 1, true, "PT1H", null, false); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -319,7 +369,7 @@ public void testReplicas() throws Exception @Test public void testLateMessageRejectionPeriod() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H")); + supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), false); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -361,7 +411,7 @@ public void testLateMessageRejectionPeriod() throws Exception */ public void testLatestOffset() throws Exception { - supervisor = getSupervisor(1, 1, false, "PT1H", null); + supervisor = getSupervisor(1, 1, false, "PT1H", null, false); addSomeEvents(1100); Capture captured = Capture.newInstance(); @@ -393,7 +443,7 @@ public void testLatestOffset() throws Exception */ public void testDatasourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); addSomeEvents(100); Capture captured = Capture.newInstance(); @@ -402,7 +452,7 @@ public void testDatasourceMetadata() throws Exception expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( - new KafkaPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) + new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) ).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true); @@ -414,7 +464,7 @@ public void testDatasourceMetadata() throws Exception KafkaIndexTask task = captured.getValue(); KafkaIOConfig taskConfig = task.getIOConfig(); - Assert.assertEquals(String.format("sequenceName-0", DATASOURCE), taskConfig.getBaseSequenceName()); + Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); @@ -423,14 +473,14 @@ public void testDatasourceMetadata() throws Exception @Test(expected = ISE.class) public void testBadMetadataOffsets() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); addSomeEvents(1); expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( - new KafkaPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) + new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) ).anyTimes(); replayAll(); @@ -442,7 +492,7 @@ public void testBadMetadataOffsets() throws Exception @Test public void testKillIncompatibleTasks() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null); + supervisor = getSupervisor(2, 1, true, "PT1H", null, false); addSomeEvents(1); Task id1 = createKafkaIndexTask( // unexpected # of partitions (kill) @@ -528,7 +578,7 @@ public void testKillIncompatibleTasks() throws Exception @Test public void testKillBadPartitionAssignment() throws Exception { - supervisor = getSupervisor(1, 2, true, "PT1H", null); + supervisor = getSupervisor(1, 2, true, "PT1H", null, false); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -612,7 +662,7 @@ public void testKillBadPartitionAssignment() throws Exception @Test public void testRequeueTaskWhenFailed() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null); + supervisor = getSupervisor(2, 2, true, "PT1H", null, false); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -681,7 +731,7 @@ public void testRequeueTaskWhenFailed() throws Exception @Test public void testRequeueAdoptedTaskWhenFailed() throws Exception { - supervisor = getSupervisor(2, 1, true, "PT1H", null); + supervisor = getSupervisor(2, 1, true, "PT1H", null, false); addSomeEvents(1); DateTime now = DateTime.now(); @@ -757,7 +807,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception @Test public void testQueueNextTasksOnSuccess() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null); + supervisor = getSupervisor(2, 2, true, "PT1H", null, false); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -828,7 +878,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234); - supervisor = getSupervisor(2, 2, true, "PT1M", null); + supervisor = getSupervisor(2, 2, true, "PT1M", null, false); addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -899,7 +949,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); - Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic()); Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); @@ -911,7 +961,7 @@ public void testDiscoverExistingPublishingTask() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234); - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); addSomeEvents(1); Task task = createKafkaIndexTask( @@ -962,7 +1012,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); + Assert.assertEquals(topic, payload.getTopic()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -984,12 +1034,12 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getTopic()); Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); - Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getTopic()); + Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getTopic()); Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); @@ -1000,7 +1050,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() { final TaskLocation location = new TaskLocation("testHost", 1234); - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); addSomeEvents(1); Task task = createKafkaIndexTask( @@ -1051,7 +1101,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); + Assert.assertEquals(topic, payload.getTopic()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1073,12 +1123,12 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getTopic()); + Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getTopic()); Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); - Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getTopic()); + Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getTopic()); Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); @@ -1091,7 +1141,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145); final DateTime startTime = new DateTime(); - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -1156,7 +1206,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); + Assert.assertEquals(topic, payload.getTopic()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1176,7 +1226,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception @Test public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception { - supervisor = getSupervisor(2, 2, true, "PT1H", null); + supervisor = getSupervisor(2, 2, true, "PT1H", null, false); addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1221,7 +1271,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234); - supervisor = getSupervisor(2, 2, true, "PT1M", null); + supervisor = getSupervisor(2, 2, true, "PT1M", null, false); addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1288,7 +1338,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception { final TaskLocation location = new TaskLocation("testHost", 1234); - supervisor = getSupervisor(2, 2, true, "PT1M", null); + supervisor = getSupervisor(2, 2, true, "PT1M", null, false); addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); @@ -1361,7 +1411,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception @Test(expected = IllegalStateException.class) public void testStopNotStarted() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); supervisor.stop(false); } @@ -1373,7 +1423,7 @@ public void testStop() throws Exception taskRunner.unregisterListener(String.format("KafkaSupervisor-%s", DATASOURCE)); replayAll(); - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); supervisor.start(); supervisor.stop(false); @@ -1387,7 +1437,7 @@ public void testStopGracefully() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145); final DateTime startTime = new DateTime(); - supervisor = getSupervisor(2, 1, true, "PT1H", null); + supervisor = getSupervisor(2, 1, true, "PT1H", null, false); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -1468,7 +1518,6 @@ public void testStopGracefully() throws Exception @Test public void testResetNoTasks() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -1476,6 +1525,7 @@ public void testResetNoTasks() throws Exception taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); supervisor.start(); supervisor.runInternal(); verifyAll(); @@ -1492,7 +1542,7 @@ public void testResetNoTasks() throws Exception @Test public void testResetDataSourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -1508,17 +1558,17 @@ public void testResetDataSourceMetadata() throws Exception Capture captureDataSourceMetadata = EasyMock.newCapture(); KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( - KAFKA_TOPIC, + topic, ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L) )); KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( - KAFKA_TOPIC, + topic, ImmutableMap.of(1, 1000L, 2, 1000L) )); KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( - KAFKA_TOPIC, + topic, ImmutableMap.of(0, 1000L) )); @@ -1540,7 +1590,7 @@ public void testResetDataSourceMetadata() throws Exception @Test public void testResetNoDataSourceMetadata() throws Exception { - supervisor = getSupervisor(1, 1, true, "PT1H", null); + supervisor = getSupervisor(1, 1, true, "PT1H", null, false); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -1553,7 +1603,7 @@ public void testResetNoDataSourceMetadata() throws Exception verifyAll(); KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( - KAFKA_TOPIC, + topic, ImmutableMap.of(1, 1000L, 2, 1000L) )); @@ -1573,7 +1623,7 @@ public void testResetRunningTasks() throws Exception final TaskLocation location2 = new TaskLocation("testHost2", 145); final DateTime startTime = new DateTime(); - supervisor = getSupervisor(2, 1, true, "PT1H", null); + supervisor = getSupervisor(2, 1, true, "PT1H", null, false); addSomeEvents(1); Task id1 = createKafkaIndexTask( @@ -1653,7 +1703,7 @@ private void addSomeEvents(int numEventsPerPartition) throws Exception for (int j = 0; j < numEventsPerPartition; j++) { kafkaProducer.send( new ProducerRecord( - KAFKA_TOPIC, + topic, i, null, String.format("event-%d", j).getBytes() @@ -1669,11 +1719,12 @@ private KafkaSupervisor getSupervisor( int taskCount, boolean useEarliestOffset, String duration, - Period lateMessageRejectionPeriod + Period lateMessageRejectionPeriod, + boolean skipOffsetGaps ) { KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( - KAFKA_TOPIC, + topic, replicas, taskCount, new Period(duration), @@ -1682,7 +1733,8 @@ private KafkaSupervisor getSupervisor( new Period("PT30S"), useEarliestOffset, new Period("PT30M"), - lateMessageRejectionPeriod + lateMessageRejectionPeriod, + skipOffsetGaps ); KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(null, null) @@ -1718,12 +1770,14 @@ public KafkaIndexTaskClient build( taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, - objectMapper + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig() ) ); } - private DataSchema getDataSchema(String dataSource) + private static DataSchema getDataSchema(String dataSource) { List dimensions = new ArrayList<>(); dimensions.add(StringDimensionSchema.create("dim1")); @@ -1778,14 +1832,15 @@ private KafkaIndexTask createKafkaIndexTask( ImmutableMap.of(), true, false, - minimumMessageTime + minimumMessageTime, + false ), ImmutableMap.of(), null ); } - private class TestTaskRunnerWorkItem extends TaskRunnerWorkItem + private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem { private TaskLocation location; @@ -1803,7 +1858,7 @@ public TaskLocation getLocation() } } - private class TestableKafkaSupervisor extends KafkaSupervisor + private static class TestableKafkaSupervisor extends KafkaSupervisor { public TestableKafkaSupervisor( TaskStorage taskStorage, diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java index f63de29e014d..e90fa4c4269e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java @@ -21,8 +21,10 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import com.google.common.io.Files; import kafka.server.KafkaConfig; import kafka.server.KafkaServer; +import org.apache.commons.io.FileUtils; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -31,6 +33,7 @@ import scala.Some; import scala.collection.immutable.List$; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -44,15 +47,22 @@ public class TestBroker implements Closeable private final String zookeeperConnect; private final File directory; + private final boolean directoryCleanup; private final int id; private final Map brokerProps; private volatile KafkaServer server; - public TestBroker(String zookeeperConnect, File directory, int id, Map brokerProps) + public TestBroker( + String zookeeperConnect, + @Nullable File directory, + int id, + Map brokerProps + ) { this.zookeeperConnect = zookeeperConnect; - this.directory = directory; + this.directory = directory == null ? Files.createTempDir() : directory; + this.directoryCleanup = directory == null; this.id = id; this.brokerProps = brokerProps == null ? ImmutableMap.of() : brokerProps; } @@ -117,5 +127,8 @@ public void close() throws IOException server.shutdown(); server.awaitShutdown(); } + if (directoryCleanup) { + FileUtils.forceDelete(directory); + } } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java index 54badfcde0e7..14b1dab8fcbe 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/NamespaceLookupExtractorFactory.java @@ -232,4 +232,10 @@ public byte[] getCacheKey() readLock.unlock(); } } + + @VisibleForTesting + CacheScheduler getCacheScheduler() + { + return cacheScheduler; + } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/CacheGenerator.java similarity index 84% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/CacheGenerator.java index 2f423b68fa07..9d76a3d60e74 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/CacheGenerator.java @@ -26,25 +26,25 @@ /** * */ -public interface ExtractionNamespaceCacheFactory +public interface CacheGenerator { /** - * If the lookup source, encapsulated by this {@code ExtractionNamespaceCacheFactory}, has data newer than identified + * If the lookup source, encapsulated by this {@code CacheGenerator}, has data newer than identified * by the given {@code lastVersion} (which is null at the first run of this method, or the version from the previous * run), this method creates a new {@code CacheScheduler.VersionedCache} with {@link * CacheScheduler#createVersionedCache}, called on the given {@code scheduler}, with the version string identifying * the current version of lookup source, populates the created {@code VersionedCache} and returns it. If the lookup * source is up-to-date, this methods returns null. * - * @param namespace The ExtractionNamespace for which to populate data. - * @param id An object uniquely corresponding to the {@link CacheScheduler.Entry}, for which this populateCache() + * @param namespace The ExtractionNamespace for which to generate cache. + * @param id An object uniquely corresponding to the {@link CacheScheduler.Entry}, for which this generateCache() * method is called. Also it has the same toString() representation, that is useful for logging * @param lastVersion The version which was last cached * @param scheduler Should be used only to call {@link CacheScheduler#createVersionedCache}. * @return the new cache along with the new version, or null if the last version is up-to-date. */ @Nullable - CacheScheduler.VersionedCache populateCache( + CacheScheduler.VersionedCache generateCache( T namespace, CacheScheduler.EntryImpl id, String lastVersion, diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java index 8eb26137295e..a6152079248c 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/ExtractionNamespace.java @@ -24,8 +24,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "jdbc", value = JDBCExtractionNamespace.class), - @JsonSubTypes.Type(name = "uri", value = URIExtractionNamespace.class), + @JsonSubTypes.Type(name = "jdbc", value = JdbcExtractionNamespace.class), + @JsonSubTypes.Type(name = "uri", value = UriExtractionNamespace.class), @JsonSubTypes.Type(name = StaticMapExtractionNamespace.TYPE_NAME, value = StaticMapExtractionNamespace.class) }) /** diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JdbcExtractionNamespace.java similarity index 95% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JdbcExtractionNamespace.java index 9fb9c3c5de8c..f7f24d65cbff 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JDBCExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/JdbcExtractionNamespace.java @@ -34,7 +34,7 @@ * */ @JsonTypeName("jdbc") -public class JDBCExtractionNamespace implements ExtractionNamespace +public class JdbcExtractionNamespace implements ExtractionNamespace { @JsonProperty private final MetadataStorageConnectorConfig connectorConfig; @@ -50,7 +50,7 @@ public class JDBCExtractionNamespace implements ExtractionNamespace private final Period pollPeriod; @JsonCreator - public JDBCExtractionNamespace( + public JdbcExtractionNamespace( @NotNull @JsonProperty(value = "connectorConfig", required = true) final MetadataStorageConnectorConfig connectorConfig, @NotNull @JsonProperty(value = "table", required = true) @@ -109,7 +109,7 @@ public long getPollMs() public String toString() { return String.format( - "JDBCExtractionNamespace = { connectorConfig = { %s }, table = %s, keyColumn = %s, valueColumn = %s, tsColumn = %s, pollPeriod = %s}", + "JdbcExtractionNamespace = { connectorConfig = { %s }, table = %s, keyColumn = %s, valueColumn = %s, tsColumn = %s, pollPeriod = %s}", connectorConfig.toString(), table, keyColumn, @@ -129,7 +129,7 @@ public boolean equals(Object o) return false; } - JDBCExtractionNamespace that = (JDBCExtractionNamespace) o; + JdbcExtractionNamespace that = (JdbcExtractionNamespace) o; if (!connectorConfig.equals(that.connectorConfig)) { return false; diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java similarity index 89% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java index a7c27c66638e..e36275909c44 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/URIExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java @@ -25,15 +25,16 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - import io.druid.guice.annotations.Json; import io.druid.java.util.common.IAE; import io.druid.java.util.common.UOE; @@ -41,7 +42,6 @@ import io.druid.java.util.common.parsers.DelimitedParser; import io.druid.java.util.common.parsers.JSONParser; import io.druid.java.util.common.parsers.Parser; - import org.joda.time.Period; import javax.annotation.Nullable; @@ -59,7 +59,7 @@ * */ @JsonTypeName("uri") -public class URIExtractionNamespace implements ExtractionNamespace +public class UriExtractionNamespace implements ExtractionNamespace { @JsonProperty private final URI uri; @@ -73,7 +73,7 @@ public class URIExtractionNamespace implements ExtractionNamespace private final Period pollPeriod; @JsonCreator - public URIExtractionNamespace( + public UriExtractionNamespace( @JsonProperty(value = "uri", required = false) URI uri, @JsonProperty(value = "uriPrefix", required = false) @@ -144,7 +144,7 @@ public long getPollMs() @Override public String toString() { - return "URIExtractionNamespace{" + + return "UriExtractionNamespace{" + "uri=" + uri + ", uriPrefix=" + uriPrefix + ", namespaceParseSpec=" + namespaceParseSpec + @@ -163,7 +163,7 @@ public boolean equals(Object o) return false; } - URIExtractionNamespace that = (URIExtractionNamespace) o; + UriExtractionNamespace that = (UriExtractionNamespace) o; if (getUri() != null ? !getUri().equals(that.getUri()) : that.getUri() != null) { return false; @@ -264,7 +264,9 @@ public static class CSVFlatDataParser implements FlatDataParser public CSVFlatDataParser( @JsonProperty("columns") List columns, @JsonProperty("keyColumn") final String keyColumn, - @JsonProperty("valueColumn") final String valueColumn + @JsonProperty("valueColumn") final String valueColumn, + @JsonProperty("hasHeaderRow") boolean hasHeaderRow, + @JsonProperty("skipHeaderRows") int skipHeaderRows ) { Preconditions.checkArgument( @@ -293,12 +295,22 @@ public CSVFlatDataParser( ); this.parser = new DelegateParser( - new CSVParser(Optional.absent(), columns), + new CSVParser(Optional.absent(), columns, hasHeaderRow, skipHeaderRows), this.keyColumn, this.valueColumn ); } + @VisibleForTesting + CSVFlatDataParser( + List columns, + String keyColumn, + String valueColumn + ) + { + this(columns, keyColumn, valueColumn, false, 0); + } + @JsonProperty public List getColumns() { @@ -373,7 +385,9 @@ public TSVFlatDataParser( @JsonProperty("delimiter") String delimiter, @JsonProperty("listDelimiter") String listDelimiter, @JsonProperty("keyColumn") final String keyColumn, - @JsonProperty("valueColumn") final String valueColumn + @JsonProperty("valueColumn") final String valueColumn, + @JsonProperty("hasHeaderRow") boolean hasHeaderRow, + @JsonProperty("skipHeaderRows") int skipHeaderRows ) { Preconditions.checkArgument( @@ -382,7 +396,9 @@ public TSVFlatDataParser( ); final DelimitedParser delegate = new DelimitedParser( Optional.fromNullable(Strings.emptyToNull(delimiter)), - Optional.fromNullable(Strings.emptyToNull(listDelimiter)) + Optional.fromNullable(Strings.emptyToNull(listDelimiter)), + hasHeaderRow, + skipHeaderRows ); Preconditions.checkArgument( !(Strings.isNullOrEmpty(keyColumn) ^ Strings.isNullOrEmpty(valueColumn)), @@ -410,6 +426,18 @@ public TSVFlatDataParser( this.parser = new DelegateParser(delegate, this.keyColumn, this.valueColumn); } + @VisibleForTesting + TSVFlatDataParser( + List columns, + String delimiter, + String listDelimiter, + String keyColumn, + String valueColumn + ) + { + this(columns, delimiter, listDelimiter, keyColumn, valueColumn, false, 0); + } + @JsonProperty public List getColumns() { @@ -503,8 +531,10 @@ public JSONFlatDataParser( Preconditions.checkArgument(!Strings.isNullOrEmpty(valueFieldName), "[valueFieldName] cannot be empty"); this.keyFieldName = keyFieldName; this.valueFieldName = valueFieldName; + + // Copy jsonMapper; don't want to share canonicalization tables, etc., with the global ObjectMapper. this.parser = new DelegateParser( - new JSONParser(jsonMapper, ImmutableList.of(keyFieldName, valueFieldName)), + new JSONParser(jsonMapper.copy(), ImmutableList.of(keyFieldName, valueFieldName)), keyFieldName, valueFieldName ); @@ -561,6 +591,9 @@ public String toString() @JsonTypeName("simpleJson") public static class ObjectMapperFlatDataParser implements FlatDataParser { + private static final TypeReference> MAP_STRING_STRING = new TypeReference>() + { + }; private final Parser parser; @@ -569,17 +602,17 @@ public ObjectMapperFlatDataParser( final @JacksonInject @Json ObjectMapper jsonMapper ) { + // There's no point canonicalizing field names, we expect them to all be unique. + final JsonFactory jsonFactory = jsonMapper.getFactory().copy(); + jsonFactory.configure(JsonFactory.Feature.CANONICALIZE_FIELD_NAMES, false); + parser = new Parser() { @Override public Map parse(String input) { try { - return jsonMapper.readValue( - input, new TypeReference>() - { - } - ); + return jsonFactory.createParser(input).readValueAs(MAP_STRING_STRING); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java similarity index 85% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java index 403bbaa4112e..1a4df7f1db82 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java @@ -22,8 +22,8 @@ import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.CacheGenerator; +import io.druid.query.lookup.namespace.JdbcExtractionNamespace; import io.druid.server.lookup.namespace.cache.CacheScheduler; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; @@ -44,18 +44,17 @@ /** * */ -public final class JDBCExtractionNamespaceCacheFactory - implements ExtractionNamespaceCacheFactory +public final class JdbcCacheGenerator implements CacheGenerator { - private static final Logger LOG = new Logger(JDBCExtractionNamespaceCacheFactory.class); - private final ConcurrentMap, DBI> dbiCache = + private static final Logger LOG = new Logger(JdbcCacheGenerator.class); + private final ConcurrentMap, DBI> dbiCache = new ConcurrentHashMap<>(); @Override @Nullable - public CacheScheduler.VersionedCache populateCache( - final JDBCExtractionNamespace namespace, - final CacheScheduler.EntryImpl entryId, + public CacheScheduler.VersionedCache generateCache( + final JdbcExtractionNamespace namespace, + final CacheScheduler.EntryImpl entryId, final String lastVersion, final CacheScheduler scheduler ) @@ -132,9 +131,9 @@ public Pair map( } } - private DBI ensureDBI(CacheScheduler.EntryImpl id, JDBCExtractionNamespace namespace) + private DBI ensureDBI(CacheScheduler.EntryImpl id, JdbcExtractionNamespace namespace) { - final CacheScheduler.EntryImpl key = id; + final CacheScheduler.EntryImpl key = id; DBI dbi = null; if (dbiCache.containsKey(key)) { dbi = dbiCache.get(key); @@ -151,7 +150,7 @@ private DBI ensureDBI(CacheScheduler.EntryImpl id, JDBC return dbi; } - private Long lastUpdates(CacheScheduler.EntryImpl id, JDBCExtractionNamespace namespace) + private Long lastUpdates(CacheScheduler.EntryImpl id, JdbcExtractionNamespace namespace) { final DBI dbi = ensureDBI(id, namespace); final String table = namespace.getTable(); diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java index 72dbbd2a9ec5..0f92191d523c 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/NamespaceExtractionModule.java @@ -30,11 +30,11 @@ import io.druid.guice.PolyBind; import io.druid.initialization.DruidModule; import io.druid.query.lookup.NamespaceLookupExtractorFactory; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.JDBCExtractionNamespace; +import io.druid.query.lookup.namespace.JdbcExtractionNamespace; import io.druid.query.lookup.namespace.StaticMapExtractionNamespace; -import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.UriExtractionNamespace; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; @@ -59,7 +59,7 @@ public List getJacksonModules() ); } - public static MapBinder, ExtractionNamespaceCacheFactory> getNamespaceFactoryMapBinder( + public static MapBinder, CacheGenerator> getNamespaceFactoryMapBinder( final Binder binder ) { @@ -68,7 +68,7 @@ public static MapBinder, ExtractionNamespac new TypeLiteral>() { }, - new TypeLiteral>() + new TypeLiteral>() { } ); @@ -92,16 +92,16 @@ public void configure(Binder binder) .in(LazySingleton.class); getNamespaceFactoryMapBinder(binder) - .addBinding(JDBCExtractionNamespace.class) - .to(JDBCExtractionNamespaceCacheFactory.class) + .addBinding(JdbcExtractionNamespace.class) + .to(JdbcCacheGenerator.class) .in(LazySingleton.class); getNamespaceFactoryMapBinder(binder) - .addBinding(URIExtractionNamespace.class) - .to(URIExtractionNamespaceCacheFactory.class) + .addBinding(UriExtractionNamespace.class) + .to(UriCacheGenerator.class) .in(LazySingleton.class); getNamespaceFactoryMapBinder(binder) .addBinding(StaticMapExtractionNamespace.class) - .to(StaticMapExtractionNamespaceCacheFactory.class) + .to(StaticMapCacheGenerator.class) .in(LazySingleton.class); } } diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapCacheGenerator.java similarity index 84% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapCacheGenerator.java index c6921dd9396b..33d203c390be 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/StaticMapCacheGenerator.java @@ -19,20 +19,20 @@ package io.druid.server.lookup.namespace; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.StaticMapExtractionNamespace; import io.druid.server.lookup.namespace.cache.CacheScheduler; import javax.annotation.Nullable; import java.util.UUID; -public final class StaticMapExtractionNamespaceCacheFactory implements ExtractionNamespaceCacheFactory +public final class StaticMapCacheGenerator implements CacheGenerator { private final String version = UUID.randomUUID().toString(); @Override @Nullable - public CacheScheduler.VersionedCache populateCache( + public CacheScheduler.VersionedCache generateCache( final StaticMapExtractionNamespace namespace, final CacheScheduler.EntryImpl id, final String lastVersion, @@ -43,7 +43,7 @@ public CacheScheduler.VersionedCache populateCache( // Throwing AssertionError, because CacheScheduler doesn't suppress Errors and will stop trying to update // the cache periodically. throw new AssertionError( - "StaticMapExtractionNamespaceCacheFactory could only be configured for a namespace which is scheduled " + "StaticMapCacheGenerator could only be configured for a namespace which is scheduled " + "to be updated once, not periodically. Last version: `" + lastVersion + "`"); } CacheScheduler.VersionedCache versionedCache = scheduler.createVersionedCache(id, version); diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java similarity index 91% rename from extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java rename to extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java index 3778421c1737..8d99c5617891 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactory.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/UriCacheGenerator.java @@ -27,8 +27,8 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.RetryUtils; import io.druid.java.util.common.logger.Logger; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.CacheGenerator; +import io.druid.query.lookup.namespace.UriExtractionNamespace; import io.druid.segment.loading.URIDataPuller; import io.druid.server.lookup.namespace.cache.CacheScheduler; @@ -44,14 +44,14 @@ /** * */ -public final class URIExtractionNamespaceCacheFactory implements ExtractionNamespaceCacheFactory +public final class UriCacheGenerator implements CacheGenerator { private static final int DEFAULT_NUM_RETRIES = 3; - private static final Logger log = new Logger(URIExtractionNamespaceCacheFactory.class); + private static final Logger log = new Logger(UriCacheGenerator.class); private final Map pullers; @Inject - public URIExtractionNamespaceCacheFactory( + public UriCacheGenerator( Map pullers ) { @@ -60,9 +60,9 @@ public URIExtractionNamespaceCacheFactory( @Override @Nullable - public CacheScheduler.VersionedCache populateCache( - final URIExtractionNamespace extractionNamespace, - final CacheScheduler.EntryImpl entryId, + public CacheScheduler.VersionedCache generateCache( + final UriExtractionNamespace extractionNamespace, + final CacheScheduler.EntryImpl entryId, @Nullable final String lastVersion, final CacheScheduler scheduler ) throws Exception diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheHandler.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheHandler.java index c249661eb632..e56bf5057488 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheHandler.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheHandler.java @@ -44,6 +44,7 @@ public ConcurrentMap getCache() return cache; } + @Override public void close() { cacheManager.disposeCache(this); diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java index a33346f0db4d..73aad2173d78 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/cache/CacheScheduler.java @@ -24,10 +24,11 @@ import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.guice.LazySingleton; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import sun.misc.Cleaner; import javax.annotation.Nullable; @@ -60,6 +61,7 @@ * entry.close(); // close the last VersionedCache and unschedule future updates * } */ +@LazySingleton public final class CacheScheduler { private static final Logger log = new Logger(CacheScheduler.class); @@ -68,9 +70,9 @@ public final class Entry implements AutoCloseable { private final EntryImpl impl; - private Entry(final T namespace, final ExtractionNamespaceCacheFactory cachePopulator) + private Entry(final T namespace, final CacheGenerator cacheGenerator) { - impl = new EntryImpl<>(namespace, this, cachePopulator); + impl = new EntryImpl<>(namespace, this, cacheGenerator); } /** @@ -141,18 +143,18 @@ public class EntryImpl implements AutoCloseable { private final AtomicReference cacheStateHolder = new AtomicReference(NoCache.CACHE_NOT_INITIALIZED); private final Future updaterFuture; private final Cleaner entryCleaner; - private final ExtractionNamespaceCacheFactory cachePopulator; + private final CacheGenerator cacheGenerator; private final UpdateCounter updateCounter = new UpdateCounter(); private final CountDownLatch startLatch = new CountDownLatch(1); - private EntryImpl(final T namespace, final Entry entry, final ExtractionNamespaceCacheFactory cachePopulator) + private EntryImpl(final T namespace, final Entry entry, final CacheGenerator cacheGenerator) { try { this.namespace = namespace; this.asString = String.format("namespace [%s] : %s", namespace, super.toString()); this.updaterFuture = schedule(namespace); this.entryCleaner = createCleaner(entry); - this.cachePopulator = cachePopulator; + this.cacheGenerator = cacheGenerator; activeEntries.incrementAndGet(); } finally { @@ -219,7 +221,7 @@ private void tryUpdateCache(String currentVersion) throws Exception boolean updatedCacheSuccessfully = false; VersionedCache newVersionedCache = null; try { - newVersionedCache = cachePopulator.populateCache(namespace, this, currentVersion, CacheScheduler.this + newVersionedCache = cacheGenerator.generateCache(namespace, this, currentVersion, CacheScheduler.this ); if (newVersionedCache != null) { CacheState previousCacheState = swapCacheState(newVersionedCache); @@ -406,7 +408,7 @@ public void close() } } - private final Map, ExtractionNamespaceCacheFactory> namespacePopulatorMap; + private final Map, CacheGenerator> namespaceGeneratorMap; private final NamespaceExtractionCacheManager cacheManager; private final AtomicLong updatesStarted = new AtomicLong(0); private final AtomicInteger activeEntries = new AtomicInteger(); @@ -414,11 +416,11 @@ public void close() @Inject public CacheScheduler( final ServiceEmitter serviceEmitter, - final Map, ExtractionNamespaceCacheFactory> namespacePopulatorMap, + final Map, CacheGenerator> namespaceGeneratorMap, NamespaceExtractionCacheManager cacheManager ) { - this.namespacePopulatorMap = namespacePopulatorMap; + this.namespaceGeneratorMap = namespaceGeneratorMap; this.cacheManager = cacheManager; cacheManager.scheduledExecutorService().scheduleAtFixedRate( new Runnable() @@ -450,7 +452,7 @@ public void run() } /** - * This method should be used from {@link ExtractionNamespaceCacheFactory#populateCache} implementations, to obtain + * This method should be used from {@link io.druid.query.lookup.namespace.CacheGenerator#generateCache} implementations, to obtain * a {@link VersionedCache} to be returned. * * @param entryId an object uniquely corresponding to the {@link CacheScheduler.Entry}, for which VersionedCache is @@ -500,11 +502,10 @@ public Entry scheduleAndWait(ExtractionNamespace namespace, long waitForFirstRun public Entry schedule(final T namespace) { - final ExtractionNamespaceCacheFactory populator = - (ExtractionNamespaceCacheFactory) namespacePopulatorMap.get(namespace.getClass()); - if (populator == null) { - throw new ISE("Cannot find populator for namespace [%s]", namespace); + final CacheGenerator generator = (CacheGenerator) namespaceGeneratorMap.get(namespace.getClass()); + if (generator == null) { + throw new ISE("Cannot find generator for namespace [%s]", namespace); } - return new Entry<>(namespace, populator); + return new Entry<>(namespace, generator); } } diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 5af474830aab..463235d69239 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -31,7 +31,6 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; - import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Json; @@ -40,10 +39,10 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.URIExtractionNamespace; +import io.druid.query.lookup.namespace.UriExtractionNamespace; import io.druid.server.DruidNode; -import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.CacheScheduler; +import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; import org.easymock.EasyMock; import org.easymock.IExpectationSetters; import org.joda.time.Period; @@ -95,10 +94,13 @@ public Object findInjectableValue( Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance ) { - if ("io.druid.server.lookup.namespace.cache.CacheScheduler".equals(valueId)) { + if (CacheScheduler.class.getName().equals(valueId)) { return scheduler; + } else if (ObjectMapper.class.getName().equals(valueId)) { + return mapper; + } else { + return null; } - return null; } } ); @@ -107,10 +109,10 @@ public Object findInjectableValue( @Test public void testSimpleSerde() throws Exception { - final URIExtractionNamespace uriExtractionNamespace = new URIExtractionNamespace( + final UriExtractionNamespace uriExtractionNamespace = new UriExtractionNamespace( temporaryFolder.newFolder().toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), + new UriExtractionNamespace.ObjectMapperFlatDataParser(mapper), Period.millis(0), null @@ -440,21 +442,7 @@ public long getPollMs() @Test public void testSerDe() throws Exception { - final Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bindInstance( - binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) - ); - } - } - ) - ); + final Injector injector = makeInjector(); final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); final String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }"; @@ -467,7 +455,7 @@ public void configure(Binder binder) LookupExtractorFactory.class ))); Assert.assertEquals( - URIExtractionNamespace.class, + UriExtractionNamespace.class, namespaceLookupExtractorFactory.getExtractionNamespace().getClass() ); Assert.assertFalse(namespaceLookupExtractorFactory.replaces(mapper.readValue(str, LookupExtractorFactory.class))); @@ -484,21 +472,7 @@ public void configure(Binder binder) @Test public void testSimpleIntrospectionHandler() throws Exception { - final Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bindInstance( - binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) - ); - } - } - ) - ); + final Injector injector = makeInjector(); final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); final String str = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"staticMap\", \"map\": {\"foo\":\"bar\"} }, \"firstCacheTimeout\":10000 }"; @@ -524,6 +498,40 @@ public void configure(Binder binder) } } + @Test + public void testSingletonCacheScheduler() throws Exception + { + final Injector injector = makeInjector(); + final ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + mapper.registerSubtypes(NamespaceLookupExtractorFactory.class); + final String str1 = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"staticMap\", \"map\": {\"foo\":\"bar\"} }, \"firstCacheTimeout\":10000 }"; + final NamespaceLookupExtractorFactory factory1 = + (NamespaceLookupExtractorFactory) mapper.readValue(str1, LookupExtractorFactory.class); + final String str2 = "{ \"type\": \"cachedNamespace\", \"extractionNamespace\": { \"type\": \"uri\", \"uriPrefix\": \"s3://bucket/prefix/\", \"fileRegex\": \"foo.*\\\\.gz\", \"namespaceParseSpec\": { \"format\": \"customJson\", \"keyFieldName\": \"someKey\", \"valueFieldName\": \"someVal\" }, \"pollPeriod\": \"PT5M\" } } }"; + final NamespaceLookupExtractorFactory factory2 = + (NamespaceLookupExtractorFactory) mapper.readValue(str2, LookupExtractorFactory.class); + Assert.assertTrue(factory1.getCacheScheduler() == factory2.getCacheScheduler()); + } + + private Injector makeInjector() + { + return Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null) + ); + } + } + ) + ); + } + @Test public void testExceptionalIntrospectionHandler() throws Exception { diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java index 590f5819876b..e636d66bdc59 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/JSONFlatDataParserTest.java @@ -92,7 +92,7 @@ public CharSequence apply(Map input) @Test public void testSimpleParse() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + final UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( MAPPER, "key", "val" @@ -106,7 +106,7 @@ public void testSimpleParse() throws Exception @Test public void testParseWithNullValues() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + final UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( MAPPER, "key", "otherVal" @@ -120,7 +120,7 @@ public void testParseWithNullValues() throws Exception @Test public void testParseWithEmptyValues() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + final UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( MAPPER, "key", "canBeEmpty" @@ -134,7 +134,7 @@ public void testParseWithEmptyValues() throws Exception @Test public void testFailParseOnKeyMissing() throws Exception { - final URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + final UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( MAPPER, "keyWHOOPS", "val" diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/UriExtractionNamespaceTest.java similarity index 85% rename from extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/UriExtractionNamespaceTest.java index 6d620b73e0b1..460e5caee35f 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/URIExtractionNamespaceTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/namespace/UriExtractionNamespaceTest.java @@ -40,7 +40,7 @@ /** * */ -public class URIExtractionNamespaceTest +public class UriExtractionNamespaceTest { public static ObjectMapper registerTypes( final ObjectMapper mapper @@ -62,7 +62,7 @@ public void configure(Binder binder) ) ) ) - ).registerSubtypes(URIExtractionNamespace.class, URIExtractionNamespace.FlatDataParser.class); + ).registerSubtypes(UriExtractionNamespace.class, UriExtractionNamespace.FlatDataParser.class); final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); mapper.setAnnotationIntrospectors( @@ -79,7 +79,7 @@ public void configure(Binder binder) @Test public void testCSV() { - URIExtractionNamespace.CSVFlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( + UriExtractionNamespace.CSVFlatDataParser parser = new UriExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", @@ -92,7 +92,7 @@ public void testCSV() @Test(expected = IllegalArgumentException.class) public void testBadCSV() { - URIExtractionNamespace.CSVFlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( + UriExtractionNamespace.CSVFlatDataParser parser = new UriExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", @@ -105,7 +105,7 @@ public void testBadCSV() @Test(expected = NullPointerException.class) public void testBadCSV2() { - URIExtractionNamespace.CSVFlatDataParser parser = new URIExtractionNamespace.CSVFlatDataParser( + UriExtractionNamespace.CSVFlatDataParser parser = new UriExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", @@ -118,7 +118,7 @@ public void testBadCSV2() @Test public void testTSV() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + UriExtractionNamespace.TSVFlatDataParser parser = new UriExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3"), "|", null, "col2", @@ -130,7 +130,7 @@ public void testTSV() @Test public void testWithListDelimiterTSV() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + UriExtractionNamespace.TSVFlatDataParser parser = new UriExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3"), "\\u0001", "\\u0002", "col2", @@ -142,7 +142,7 @@ public void testWithListDelimiterTSV() @Test(expected = IllegalArgumentException.class) public void testBadTSV() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + UriExtractionNamespace.TSVFlatDataParser parser = new UriExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3fdsfds"), ",", null, "col2", @@ -156,7 +156,7 @@ public void testBadTSV() @Test(expected = NullPointerException.class) public void testBadTSV2() { - URIExtractionNamespace.TSVFlatDataParser parser = new URIExtractionNamespace.TSVFlatDataParser( + UriExtractionNamespace.TSVFlatDataParser parser = new UriExtractionNamespace.TSVFlatDataParser( ImmutableList.of("col1", "col2", "col3"), ",", null, "col2", @@ -171,7 +171,7 @@ public void testJSONFlatDataParser() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( new ObjectMapper(), keyField, valueField @@ -195,7 +195,7 @@ public void testJSONFlatDataParserBad() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( new ObjectMapper(), keyField, valueField @@ -218,7 +218,7 @@ public void testJSONFlatDataParserBad2() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( registerTypes(new ObjectMapper()), null, valueField @@ -241,7 +241,7 @@ public void testJSONFlatDataParserBad3() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( registerTypes(new ObjectMapper()), keyField, null @@ -264,7 +264,7 @@ public void testJSONFlatDataParserBad4() { final String keyField = "keyField"; final String valueField = "valueField"; - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( registerTypes(new ObjectMapper()), "", "" @@ -285,7 +285,7 @@ public void testJSONFlatDataParserBad4() @Test public void testObjectMapperFlatDataParser() { - URIExtractionNamespace.ObjectMapperFlatDataParser parser = new URIExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespace.ObjectMapperFlatDataParser parser = new UriExtractionNamespace.ObjectMapperFlatDataParser( registerTypes(new ObjectMapper()) ); Assert.assertEquals(ImmutableMap.of("B", "C"), parser.getParser().parse("{\"B\":\"C\"}")); @@ -295,22 +295,22 @@ public void testObjectMapperFlatDataParser() public void testSimpleJSONSerDe() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); - for (URIExtractionNamespace.FlatDataParser parser : ImmutableList.of( - new URIExtractionNamespace.CSVFlatDataParser( + for (UriExtractionNamespace.FlatDataParser parser : ImmutableList.of( + new UriExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", "col3" ), "col2", "col3" ), - new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), - new URIExtractionNamespace.JSONFlatDataParser(mapper, "keyField", "valueField"), - new URIExtractionNamespace.TSVFlatDataParser(ImmutableList.of("A", "B"), ",", null, "A", "B") + new UriExtractionNamespace.ObjectMapperFlatDataParser(mapper), + new UriExtractionNamespace.JSONFlatDataParser(mapper, "keyField", "valueField"), + new UriExtractionNamespace.TSVFlatDataParser(ImmutableList.of("A", "B"), ",", null, "A", "B") )) { final String str = mapper.writeValueAsString(parser); - final URIExtractionNamespace.FlatDataParser parser2 = mapper.readValue( + final UriExtractionNamespace.FlatDataParser parser2 = mapper.readValue( str, - URIExtractionNamespace.FlatDataParser.class + UriExtractionNamespace.FlatDataParser.class ); Assert.assertEquals(str, mapper.writeValueAsString(parser2)); } @@ -320,17 +320,17 @@ public void testSimpleJSONSerDe() throws IOException public void testSimpleToString() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); - for (URIExtractionNamespace.FlatDataParser parser : ImmutableList.of( - new URIExtractionNamespace.CSVFlatDataParser( + for (UriExtractionNamespace.FlatDataParser parser : ImmutableList.of( + new UriExtractionNamespace.CSVFlatDataParser( ImmutableList.of( "col1", "col2", "col3" ), "col2", "col3" ), - new URIExtractionNamespace.ObjectMapperFlatDataParser(mapper), - new URIExtractionNamespace.JSONFlatDataParser(mapper, "keyField", "valueField"), - new URIExtractionNamespace.TSVFlatDataParser(ImmutableList.of("A", "B"), ",", null, "A", "B") + new UriExtractionNamespace.ObjectMapperFlatDataParser(mapper), + new UriExtractionNamespace.JSONFlatDataParser(mapper, "keyField", "valueField"), + new UriExtractionNamespace.TSVFlatDataParser(ImmutableList.of("A", "B"), ",", null, "A", "B") )) { Assert.assertFalse(parser.toString().contains("@")); } @@ -340,13 +340,13 @@ public void testSimpleToString() throws IOException public void testMatchedJson() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); - URIExtractionNamespace namespace = mapper.readValue( + UriExtractionNamespace namespace = mapper.readValue( "{\"type\":\"uri\", \"uriPrefix\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\", \"namespace\":\"testNamespace\"}", - URIExtractionNamespace.class + UriExtractionNamespace.class ); Assert.assertEquals( - URIExtractionNamespace.ObjectMapperFlatDataParser.class.getCanonicalName(), + UriExtractionNamespace.ObjectMapperFlatDataParser.class.getCanonicalName(), namespace.getNamespaceParseSpec().getClass().getCanonicalName() ); Assert.assertEquals("file:/foo", namespace.getUriPrefix().toString()); @@ -358,13 +358,13 @@ public void testMatchedJson() throws IOException public void testExplicitJson() throws IOException { final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); - URIExtractionNamespace namespace = mapper.readValue( + UriExtractionNamespace namespace = mapper.readValue( "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\"}", - URIExtractionNamespace.class + UriExtractionNamespace.class ); Assert.assertEquals( - URIExtractionNamespace.ObjectMapperFlatDataParser.class.getCanonicalName(), + UriExtractionNamespace.ObjectMapperFlatDataParser.class.getCanonicalName(), namespace.getNamespaceParseSpec().getClass().getCanonicalName() ); Assert.assertEquals("file:/foo", namespace.getUri().toString()); @@ -377,7 +377,7 @@ public void testExplicitJsonException() throws IOException final ObjectMapper mapper = registerTypes(new DefaultObjectMapper()); mapper.readValue( "{\"type\":\"uri\", \"uri\":\"file:/foo\", \"namespaceParseSpec\":{\"format\":\"simpleJson\"}, \"pollPeriod\":\"PT5M\", \"versionRegex\":\"a.b.c\", \"namespace\":\"testNamespace\"}", - URIExtractionNamespace.class + UriExtractionNamespace.class ); } @@ -388,7 +388,7 @@ public void testFlatDataNumeric() final String valueField = "valueField"; final int n = 341879; final String nString = String.format("%d", n); - URIExtractionNamespace.JSONFlatDataParser parser = new URIExtractionNamespace.JSONFlatDataParser( + UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser( new ObjectMapper(), keyField, valueField @@ -450,7 +450,7 @@ public void testFlatDataNumeric() @Test public void testSimpleJsonNumeric() { - final URIExtractionNamespace.ObjectMapperFlatDataParser parser = new URIExtractionNamespace.ObjectMapperFlatDataParser( + final UriExtractionNamespace.ObjectMapperFlatDataParser parser = new UriExtractionNamespace.ObjectMapperFlatDataParser( registerTypes(new DefaultObjectMapper()) ); final int n = 341879; diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java index 4dcb409484db..92af1f755c78 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/NamespacedExtractorModuleTest.java @@ -24,11 +24,11 @@ import io.druid.data.SearchableVersionedDataFinder; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.JDBCExtractionNamespace; -import io.druid.query.lookup.namespace.URIExtractionNamespace; -import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.namespace.JdbcExtractionNamespace; +import io.druid.query.lookup.namespace.UriExtractionNamespace; +import io.druid.query.lookup.namespace.UriExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.lookup.namespace.cache.CacheScheduler; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; @@ -51,7 +51,7 @@ */ public class NamespacedExtractorModuleTest { - private static final ObjectMapper mapper = URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()); + private static final ObjectMapper mapper = UriExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()); private CacheScheduler scheduler; private Lifecycle lifecycle; @@ -61,16 +61,16 @@ public class NamespacedExtractorModuleTest @Before public void setUp() throws Exception { - final Map, ExtractionNamespaceCacheFactory> factoryMap = - ImmutableMap., ExtractionNamespaceCacheFactory>of( - URIExtractionNamespace.class, - new URIExtractionNamespaceCacheFactory( + final Map, CacheGenerator> factoryMap = + ImmutableMap., CacheGenerator>of( + UriExtractionNamespace.class, + new UriCacheGenerator( ImmutableMap.of( "file", new LocalFileTimestampVersionFinder() ) ), - JDBCExtractionNamespace.class, new JDBCExtractionNamespaceCacheFactory() + JdbcExtractionNamespace.class, new JdbcCacheGenerator() ); lifecycle = new Lifecycle(); lifecycle.start(); @@ -95,19 +95,19 @@ public void testNewTask() throws Exception try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } - final URIExtractionNamespaceCacheFactory factory = new URIExtractionNamespaceCacheFactory( + final UriCacheGenerator factory = new UriCacheGenerator( ImmutableMap.of("file", new LocalFileTimestampVersionFinder()) ); - final URIExtractionNamespace namespace = new URIExtractionNamespace( + final UriExtractionNamespace namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) ), new Period(0), null ); - CacheScheduler.VersionedCache versionedCache = factory.populateCache(namespace, null, null, scheduler); + CacheScheduler.VersionedCache versionedCache = factory.generateCache(namespace, null, null, scheduler); Assert.assertNotNull(versionedCache); Map map = versionedCache.getCache(); Assert.assertEquals("bar", map.get("foo")); @@ -121,10 +121,10 @@ public void testListNamespaces() throws Exception try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } - final URIExtractionNamespace namespace = new URIExtractionNamespace( + final UriExtractionNamespace namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser(URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper())), + new UriExtractionNamespace.ObjectMapperFlatDataParser(UriExtractionNamespaceTest.registerTypes(new DefaultObjectMapper())), new Period(0), null ); @@ -142,11 +142,11 @@ public void testDeleteNamespaces() throws Exception try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } - final URIExtractionNamespace namespace = new URIExtractionNamespace( + final UriExtractionNamespace namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) ), new Period(0), null @@ -163,11 +163,11 @@ public void testNewUpdate() throws Exception try (OutputStreamWriter out = new FileWriter(tmpFile)) { out.write(mapper.writeValueAsString(ImmutableMap.of("foo", "bar"))); } - final URIExtractionNamespace namespace = new URIExtractionNamespace( + final UriExtractionNamespace namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new DefaultObjectMapper()) ), new Period(0), null diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapCacheGeneratorTest.java similarity index 80% rename from extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapCacheGeneratorTest.java index 50a87f1a1903..43792b01f56f 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/StaticMapCacheGeneratorTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableMap; import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.StaticMapExtractionNamespace; import io.druid.server.lookup.namespace.cache.CacheScheduler; @@ -35,7 +35,7 @@ import java.util.Collections; import java.util.Map; -public class StaticMapExtractionNamespaceCacheFactoryTest +public class StaticMapCacheGeneratorTest { private static final Map MAP = ImmutableMap.builder().put("foo", "bar").build(); @@ -50,7 +50,7 @@ public void setup() throws Exception NoopServiceEmitter noopServiceEmitter = new NoopServiceEmitter(); scheduler = new CacheScheduler( noopServiceEmitter, - Collections., ExtractionNamespaceCacheFactory>emptyMap(), + Collections., CacheGenerator>emptyMap(), new OnHeapNamespaceExtractionCacheManager(lifecycle, noopServiceEmitter) ); } @@ -62,11 +62,11 @@ public void tearDown() } @Test - public void testSimplePopulator() throws Exception + public void testSimpleGenerator() throws Exception { - final StaticMapExtractionNamespaceCacheFactory factory = new StaticMapExtractionNamespaceCacheFactory(); + final StaticMapCacheGenerator factory = new StaticMapCacheGenerator(); final StaticMapExtractionNamespace namespace = new StaticMapExtractionNamespace(MAP); - CacheScheduler.VersionedCache versionedCache = factory.populateCache(namespace, null, null, scheduler); + CacheScheduler.VersionedCache versionedCache = factory.generateCache(namespace, null, null, scheduler); Assert.assertNotNull(versionedCache); Assert.assertEquals(factory.getVersion(), versionedCache.getVersion()); Assert.assertEquals(MAP, versionedCache.getCache()); @@ -76,8 +76,8 @@ public void testSimplePopulator() throws Exception @Test(expected = AssertionError.class) public void testNonNullLastVersionCausesAssertionError() { - final StaticMapExtractionNamespaceCacheFactory factory = new StaticMapExtractionNamespaceCacheFactory(); + final StaticMapCacheGenerator factory = new StaticMapCacheGenerator(); final StaticMapExtractionNamespace namespace = new StaticMapExtractionNamespace(MAP); - factory.populateCache(namespace, null, factory.getVersion(), scheduler); + factory.generateCache(namespace, null, factory.getVersion(), scheduler); } } diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/UriCacheGeneratorTest.java similarity index 84% rename from extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/UriCacheGeneratorTest.java index 5699ccde71dd..630463de1ade 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/URIExtractionNamespaceCacheFactoryTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/UriCacheGeneratorTest.java @@ -29,14 +29,14 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.UOE; import io.druid.java.util.common.lifecycle.Lifecycle; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.lookup.namespace.URIExtractionNamespace; -import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.namespace.UriExtractionNamespace; +import io.druid.query.lookup.namespace.UriExtractionNamespaceTest; import io.druid.segment.loading.LocalFileTimestampVersionFinder; import io.druid.server.lookup.namespace.cache.CacheScheduler; import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager; -import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManagerExecutorsTest; +import io.druid.server.lookup.namespace.cache.CacheSchedulerTest; import io.druid.server.lookup.namespace.cache.OffHeapNamespaceExtractionCacheManager; import io.druid.server.lookup.namespace.cache.OnHeapNamespaceExtractionCacheManager; import io.druid.server.metrics.NoopServiceEmitter; @@ -76,7 +76,7 @@ * */ @RunWith(Parameterized.class) -public class URIExtractionNamespaceCacheFactoryTest +public class UriCacheGeneratorTest { private static final String FAKE_SCHEME = "wabblywoo"; private static final Map FINDERS = ImmutableMap.of( @@ -214,7 +214,7 @@ public Object[] next() { if (cacheManagerCreatorsIt.hasNext()) { Function cacheManagerCreator = cacheManagerCreatorsIt.next(); - return new Object[]{ compressions[0], compressions[1], cacheManagerCreator }; + return new Object[]{compressions[0], compressions[1], cacheManagerCreator}; } else { cacheManagerCreatorsIt = cacheManagerCreators.iterator(); compressions = compressionIt.next(); @@ -232,13 +232,13 @@ public void remove() }; } - public URIExtractionNamespaceCacheFactoryTest( + public UriCacheGeneratorTest( String suffix, Function outStreamSupplier, Function cacheManagerCreator ) throws Exception { - final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap = new HashMap<>(); + final Map, CacheGenerator> namespaceFunctionFactoryMap = new HashMap<>(); this.suffix = suffix; this.outStreamSupplier = outStreamSupplier; this.lifecycle = new Lifecycle(); @@ -248,9 +248,9 @@ public URIExtractionNamespaceCacheFactoryTest( cacheManagerCreator.apply(lifecycle) ); namespaceFunctionFactoryMap.put( - URIExtractionNamespace.class, + UriExtractionNamespace.class, - new URIExtractionNamespaceCacheFactory(FINDERS) + new UriCacheGenerator(FINDERS) ); } @@ -263,8 +263,8 @@ public URIExtractionNamespaceCacheFactoryTest( private CacheScheduler scheduler; private File tmpFile; private File tmpFileParent; - private URIExtractionNamespaceCacheFactory populator; - private URIExtractionNamespace namespace; + private UriCacheGenerator generator; + private UriExtractionNamespace namespace; @Before public void setUp() throws Exception @@ -275,26 +275,25 @@ public void setUp() throws Exception Assert.assertTrue(tmpFileParent.isDirectory()); tmpFile = Files.createTempFile(tmpFileParent.toPath(), "druidTestURIExtractionNS", suffix).toFile(); final ObjectMapper mapper = new DefaultObjectMapper(); - try (OutputStream ostream = outStreamSupplier.apply(tmpFile)) { - try (OutputStreamWriter out = new OutputStreamWriter(ostream)) { - out.write(mapper.writeValueAsString(ImmutableMap.of( - "boo", - "bar", - "foo", - "bar", - "", - "MissingValue", - "emptyString", - "" - ))); - } + try (OutputStream ostream = outStreamSupplier.apply(tmpFile); + OutputStreamWriter out = new OutputStreamWriter(ostream)) { + out.write(mapper.writeValueAsString(ImmutableMap.of( + "boo", + "bar", + "foo", + "bar", + "", + "MissingValue", + "emptyString", + "" + ))); } - populator = new URIExtractionNamespaceCacheFactory(FINDERS); - namespace = new URIExtractionNamespace( + generator = new UriCacheGenerator(FINDERS); + namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new ObjectMapper()) ), new Period(0), null @@ -312,7 +311,7 @@ public void simpleTest() throws IOException, ExecutionException, InterruptedExce { Assert.assertEquals(0, scheduler.getActiveEntries()); CacheScheduler.Entry entry = scheduler.schedule(namespace); - NamespaceExtractionCacheManagerExecutorsTest.waitFor(entry); + CacheSchedulerTest.waitFor(entry); Map map = entry.getCache(); Assert.assertEquals("bar", map.get("foo")); Assert.assertEquals(null, map.get("baz")); @@ -321,7 +320,7 @@ public void simpleTest() throws IOException, ExecutionException, InterruptedExce @Test public void simpleTestRegex() throws IOException, ExecutionException, InterruptedException { - final URIExtractionNamespace namespace = new URIExtractionNamespace( + final UriExtractionNamespace namespace = new UriExtractionNamespace( null, Paths.get(this.namespace.getUri()).getParent().toUri(), Pattern.quote(Paths.get(this.namespace.getUri()).getFileName().toString()), @@ -330,7 +329,7 @@ public void simpleTestRegex() throws IOException, ExecutionException, Interrupte null ); CacheScheduler.Entry entry = scheduler.schedule(namespace); - NamespaceExtractionCacheManagerExecutorsTest.waitFor(entry); + CacheSchedulerTest.waitFor(entry); Map map = entry.getCache(); Assert.assertNotNull(map); Assert.assertEquals("bar", map.get("foo")); @@ -343,11 +342,11 @@ public void simplePileONamespacesTest() throws InterruptedException final int size = 128; List entries = new ArrayList<>(size); for (int i = 0; i < size; ++i) { - URIExtractionNamespace namespace = new URIExtractionNamespace( + UriExtractionNamespace namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new ObjectMapper()) ), new Period(0), null @@ -355,7 +354,7 @@ public void simplePileONamespacesTest() throws InterruptedException CacheScheduler.Entry entry = scheduler.schedule(namespace); entries.add(entry); - NamespaceExtractionCacheManagerExecutorsTest.waitFor(entry); + CacheSchedulerTest.waitFor(entry); } for (CacheScheduler.Entry entry : entries) { @@ -372,7 +371,7 @@ public void testLoadOnlyOnce() throws Exception { Assert.assertEquals(0, scheduler.getActiveEntries()); - CacheScheduler.VersionedCache versionedCache = populator.populateCache(namespace, null, null, scheduler); + CacheScheduler.VersionedCache versionedCache = generator.generateCache(namespace, null, null, scheduler); Assert.assertNotNull(versionedCache); Map map = versionedCache.getCache(); Assert.assertEquals("bar", map.get("foo")); @@ -380,13 +379,13 @@ public void testLoadOnlyOnce() throws Exception String version = versionedCache.getVersion(); Assert.assertNotNull(version); - Assert.assertNull(populator.populateCache(namespace, null, version, scheduler)); + Assert.assertNull(generator.generateCache(namespace, null, version, scheduler)); } @Test(expected = FileNotFoundException.class) public void testMissing() throws Exception { - URIExtractionNamespace badNamespace = new URIExtractionNamespace( + UriExtractionNamespace badNamespace = new UriExtractionNamespace( namespace.getUri(), null, null, namespace.getNamespaceParseSpec(), @@ -394,13 +393,13 @@ public void testMissing() throws Exception null ); Assert.assertTrue(new File(namespace.getUri()).delete()); - populator.populateCache(badNamespace, null, null, scheduler); + generator.generateCache(badNamespace, null, null, scheduler); } @Test(expected = FileNotFoundException.class) public void testMissingRegex() throws Exception { - URIExtractionNamespace badNamespace = new URIExtractionNamespace( + UriExtractionNamespace badNamespace = new UriExtractionNamespace( null, Paths.get(namespace.getUri()).getParent().toUri(), Pattern.quote(Paths.get(namespace.getUri()).getFileName().toString()), @@ -409,13 +408,13 @@ public void testMissingRegex() throws Exception null ); Assert.assertTrue(new File(namespace.getUri()).delete()); - populator.populateCache(badNamespace, null, null, scheduler); + generator.generateCache(badNamespace, null, null, scheduler); } @Test(expected = IAE.class) public void testExceptionalCreationDoubleURI() { - new URIExtractionNamespace( + new UriExtractionNamespace( namespace.getUri(), namespace.getUri(), null, @@ -428,7 +427,7 @@ public void testExceptionalCreationDoubleURI() @Test(expected = IAE.class) public void testExceptionalCreationURIWithPattern() { - new URIExtractionNamespace( + new UriExtractionNamespace( namespace.getUri(), null, "", @@ -441,7 +440,7 @@ public void testExceptionalCreationURIWithPattern() @Test(expected = IAE.class) public void testExceptionalCreationURIWithLegacyPattern() { - new URIExtractionNamespace( + new UriExtractionNamespace( namespace.getUri(), null, null, @@ -454,7 +453,7 @@ public void testExceptionalCreationURIWithLegacyPattern() @Test(expected = IAE.class) public void testLegacyMix() { - new URIExtractionNamespace( + new UriExtractionNamespace( null, namespace.getUri(), "", @@ -468,7 +467,7 @@ public void testLegacyMix() @Test(expected = IAE.class) public void testBadPattern() { - new URIExtractionNamespace( + new UriExtractionNamespace( null, namespace.getUri(), "[", @@ -481,7 +480,7 @@ public void testBadPattern() @Test public void testWeirdSchemaOnExactURI() throws Exception { - final URIExtractionNamespace extractionNamespace = new URIExtractionNamespace( + final UriExtractionNamespace extractionNamespace = new UriExtractionNamespace( new URI( FAKE_SCHEME, namespace.getUri().getUserInfo(), @@ -497,19 +496,19 @@ public void testWeirdSchemaOnExactURI() throws Exception Period.millis((int) namespace.getPollMs()), null ); - Assert.assertNotNull(populator.populateCache(extractionNamespace, null, null, scheduler)); + Assert.assertNotNull(generator.generateCache(extractionNamespace, null, null, scheduler)); } @Test(timeout = 10_000) public void testDeleteOnScheduleFail() throws Exception { Assert.assertNull(scheduler.scheduleAndWait( - new URIExtractionNamespace( + new UriExtractionNamespace( new URI("file://tmp/I_DONT_REALLY_EXIST" + UUID.randomUUID().toString()), null, null, - new URIExtractionNamespace.JSONFlatDataParser( + new UriExtractionNamespace.JSONFlatDataParser( new DefaultObjectMapper(), "key", "val" diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/CacheSchedulerTest.java similarity index 89% rename from extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/CacheSchedulerTest.java index f3904d9570bc..7d92692b195a 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagerExecutorsTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/CacheSchedulerTest.java @@ -28,10 +28,10 @@ import com.google.common.util.concurrent.MoreExecutors; import io.druid.concurrent.Execs; import io.druid.java.util.common.lifecycle.Lifecycle; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.lookup.namespace.URIExtractionNamespace; -import io.druid.query.lookup.namespace.URIExtractionNamespaceTest; +import io.druid.query.lookup.namespace.UriExtractionNamespace; +import io.druid.query.lookup.namespace.UriExtractionNamespaceTest; import io.druid.server.metrics.NoopServiceEmitter; import org.joda.time.Period; import org.junit.After; @@ -65,7 +65,7 @@ * */ @RunWith(Parameterized.class) -public class NamespaceExtractionCacheManagerExecutorsTest +public class CacheSchedulerTest { public static final Function CREATE_ON_HEAP_CACHE_MANAGER = new Function() @@ -111,7 +111,7 @@ public static void waitFor(CacheScheduler.Entry entry) throws InterruptedExcepti private CacheScheduler scheduler; private File tmpFile; - public NamespaceExtractionCacheManagerExecutorsTest( + public CacheSchedulerTest( Function createCacheManager ) { @@ -125,13 +125,13 @@ public void setUp() throws Exception lifecycle.start(); cacheManager = createCacheManager.apply(lifecycle); final Path tmpDir = temporaryFolder.newFolder().toPath(); - final ExtractionNamespaceCacheFactory cachePopulator = new - ExtractionNamespaceCacheFactory() + final CacheGenerator cacheGenerator = new + CacheGenerator() { @Override - public CacheScheduler.VersionedCache populateCache( - final URIExtractionNamespace extractionNamespace, - final CacheScheduler.EntryImpl id, + public CacheScheduler.VersionedCache generateCache( + final UriExtractionNamespace extractionNamespace, + final CacheScheduler.EntryImpl id, final String lastVersion, final CacheScheduler scheduler ) throws InterruptedException @@ -146,9 +146,9 @@ public CacheScheduler.VersionedCache populateCache( }; scheduler = new CacheScheduler( new NoopServiceEmitter(), - ImmutableMap., ExtractionNamespaceCacheFactory>of( - URIExtractionNamespace.class, - cachePopulator + ImmutableMap., CacheGenerator>of( + UriExtractionNamespace.class, + cacheGenerator ), cacheManager ); @@ -172,11 +172,11 @@ public void tearDown() @Test(timeout = 10_000) public void testSimpleSubmission() throws ExecutionException, InterruptedException { - URIExtractionNamespace namespace = new URIExtractionNamespace( + UriExtractionNamespace namespace = new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new ObjectMapper()) ), new Period(0), null @@ -194,7 +194,7 @@ public void testPeriodicUpdatesScheduled() throws ExecutionException, Interrupte final int repeatCount = 5; final long delay = 5; try { - final URIExtractionNamespace namespace = getUriExtractionNamespace(delay); + final UriExtractionNamespace namespace = getUriExtractionNamespace(delay); final long start = System.currentTimeMillis(); try (CacheScheduler.Entry entry = scheduler.schedule(namespace)) { @@ -305,7 +305,7 @@ public void testDelete() throws InterruptedException, TimeoutException, ExecutionException { final long period = 1_000L;// Give it some time between attempts to update - final URIExtractionNamespace namespace = getUriExtractionNamespace(period); + final UriExtractionNamespace namespace = getUriExtractionNamespace(period); CacheScheduler.Entry entry = scheduler.scheduleAndWait(namespace, 10_000); Assert.assertNotNull(entry); final Future future = entry.getUpdaterFuture(); @@ -332,13 +332,13 @@ public void testDelete() Assert.assertTrue(future.isDone()); } - private URIExtractionNamespace getUriExtractionNamespace(long period) + private UriExtractionNamespace getUriExtractionNamespace(long period) { - return new URIExtractionNamespace( + return new UriExtractionNamespace( tmpFile.toURI(), null, null, - new URIExtractionNamespace.ObjectMapperFlatDataParser( - URIExtractionNamespaceTest.registerTypes(new ObjectMapper()) + new UriExtractionNamespace.ObjectMapperFlatDataParser( + UriExtractionNamespaceTest.registerTypes(new ObjectMapper()) ), new Period(period), null @@ -352,7 +352,7 @@ public void testShutdown() final long period = 5L; try { - final URIExtractionNamespace namespace = getUriExtractionNamespace(period); + final UriExtractionNamespace namespace = getUriExtractionNamespace(period); try (CacheScheduler.Entry entry = scheduler.schedule(namespace)) { final Future future = entry.getUpdaterFuture(); @@ -383,7 +383,7 @@ public void testRunCount() throws InterruptedException, ExecutionException { final int numWaits = 5; try { - final URIExtractionNamespace namespace = getUriExtractionNamespace((long) 5); + final UriExtractionNamespace namespace = getUriExtractionNamespace((long) 5); try (CacheScheduler.Entry entry = scheduler.schedule(namespace)) { final Future future = entry.getUpdaterFuture(); entry.awaitNextUpdates(numWaits); diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java similarity index 92% rename from extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java rename to extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java index 03be4b4637a4..da5099e5dbf6 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JDBCExtractionNamespaceTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java @@ -31,10 +31,10 @@ import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.TestDerbyConnector; +import io.druid.query.lookup.namespace.CacheGenerator; import io.druid.query.lookup.namespace.ExtractionNamespace; -import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import io.druid.query.lookup.namespace.JDBCExtractionNamespace; -import io.druid.server.lookup.namespace.JDBCExtractionNamespaceCacheFactory; +import io.druid.query.lookup.namespace.JdbcExtractionNamespace; +import io.druid.server.lookup.namespace.JdbcCacheGenerator; import io.druid.server.metrics.NoopServiceEmitter; import org.joda.time.Period; import org.junit.After; @@ -62,11 +62,11 @@ * */ @RunWith(Parameterized.class) -public class JDBCExtractionNamespaceTest +public class JdbcExtractionNamespaceTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private static final Logger log = new Logger(JDBCExtractionNamespaceTest.class); + private static final Logger log = new Logger(JdbcExtractionNamespaceTest.class); private static final String tableName = "abstractDbRenameTest"; private static final String keyName = "keyName"; private static final String valName = "valName"; @@ -87,7 +87,7 @@ public static Collection getParameters() ); } - public JDBCExtractionNamespaceTest( + public JdbcExtractionNamespaceTest( String tsColumn ) { @@ -186,25 +186,25 @@ public void close() throws IOException NoopServiceEmitter noopServiceEmitter = new NoopServiceEmitter(); scheduler = new CacheScheduler( noopServiceEmitter, - ImmutableMap., ExtractionNamespaceCacheFactory>of( - JDBCExtractionNamespace.class, - new ExtractionNamespaceCacheFactory() + ImmutableMap., CacheGenerator>of( + JdbcExtractionNamespace.class, + new CacheGenerator() { - private final JDBCExtractionNamespaceCacheFactory delegate = - new JDBCExtractionNamespaceCacheFactory(); + private final JdbcCacheGenerator delegate = + new JdbcCacheGenerator(); @Override - public CacheScheduler.VersionedCache populateCache( - final JDBCExtractionNamespace namespace, - final CacheScheduler.EntryImpl id, + public CacheScheduler.VersionedCache generateCache( + final JdbcExtractionNamespace namespace, + final CacheScheduler.EntryImpl id, final String lastVersion, final CacheScheduler scheduler ) throws InterruptedException { updateLock.lockInterruptibly(); try { - log.debug("Running cache populator"); + log.debug("Running cache generator"); try { - return delegate.populateCache(namespace, id, lastVersion, scheduler); + return delegate.generateCache(namespace, id, lastVersion, scheduler); } finally { updates.incrementAndGet(); @@ -355,7 +355,7 @@ public void testMapping() throws ClassNotFoundException, NoSuchFieldException, IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { - final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( + final JdbcExtractionNamespace extractionNamespace = new JdbcExtractionNamespace( derbyConnectorRule.getMetadataConnectorConfig(), tableName, keyName, @@ -364,7 +364,7 @@ public void testMapping() new Period(0) ); try (CacheScheduler.Entry entry = scheduler.schedule(extractionNamespace)) { - NamespaceExtractionCacheManagerExecutorsTest.waitFor(entry); + CacheSchedulerTest.waitFor(entry); final Map map = entry.getCache(); for (Map.Entry e : renames.entrySet()) { @@ -403,7 +403,7 @@ public void testFindNew() private CacheScheduler.Entry ensureEntry() throws NoSuchFieldException, IllegalAccessException, InterruptedException { - final JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( + final JdbcExtractionNamespace extractionNamespace = new JdbcExtractionNamespace( derbyConnectorRule.getMetadataConnectorConfig(), tableName, keyName, diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java index 472bb437b248..7c7013c1f591 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/NamespaceExtractionCacheManagersTest.java @@ -47,8 +47,8 @@ public class NamespaceExtractionCacheManagersTest public static Collection data() { return Arrays.asList(new Object[][]{ - {NamespaceExtractionCacheManagerExecutorsTest.CREATE_ON_HEAP_CACHE_MANAGER}, - {NamespaceExtractionCacheManagerExecutorsTest.CREATE_OFF_HEAP_CACHE_MANAGER} + {CacheSchedulerTest.CREATE_ON_HEAP_CACHE_MANAGER}, + {CacheSchedulerTest.CREATE_OFF_HEAP_CACHE_MANAGER} }); } diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/LoadingCache.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/LoadingCache.java index 2abde26b6f0d..5a32ea934705 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/LoadingCache.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/LoadingCache.java @@ -128,5 +128,6 @@ public interface LoadingCache extends Closeable * the namespace deletion event should call this method to clean up resources. */ + @Override void close(); } diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml new file mode 100644 index 000000000000..01173413f238 --- /dev/null +++ b/extensions-core/protobuf-extensions/pom.xml @@ -0,0 +1,123 @@ + + + + + 4.0.0 + + io.druid.extensions + druid-protobuf-extensions + druid-protobuf-extensions + druid-protobuf-extensions + + + druid + io.druid + 0.10.1-SNAPSHOT + ../../pom.xml + + + + 3.2.0 + + + + + io.druid + druid-common + ${project.parent.version} + provided + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + com.google.protobuf + protobuf-java-util + ${protobuf.version} + + + com.github.os72 + protobuf-dynamic + 0.9.3 + + + + junit + junit + test + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + com.google.protobuf + shaded.com.google.protobuf + + + + + + package + + shade + + + + + + org.apache.maven.plugins + maven-resources-plugin + 3.0.2 + + + desc + + + + + + + + strict + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + -Xep:MissingOverride:WARN + + + + + + + + diff --git a/processing/src/main/java/io/druid/segment/data/ListBasedIndexedInts.java b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java similarity index 55% rename from processing/src/main/java/io/druid/segment/data/ListBasedIndexedInts.java rename to extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java index a76c5f2fe515..d5b7055a1405 100644 --- a/processing/src/main/java/io/druid/segment/data/ListBasedIndexedInts.java +++ b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufExtensionsModule.java @@ -17,48 +17,32 @@ * under the License. */ -package io.druid.segment.data; +package io.druid.data.input.protobuf; -import it.unimi.dsi.fastutil.ints.IntIterator; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; -import java.io.IOException; +import java.util.Arrays; import java.util.List; -/** - */ -public class ListBasedIndexedInts implements IndexedInts +public class ProtobufExtensionsModule implements DruidModule { - private final List expansion; - - public ListBasedIndexedInts(List expansion) {this.expansion = expansion;} @Override - public int size() + public List getJacksonModules() { - return expansion.size(); + return Arrays.asList( + new SimpleModule("ProtobufInputRowParserModule") + .registerSubtypes( + new NamedType(ProtobufInputRowParser.class, "protobuf") + ) + ); } @Override - public int get(int index) - { - return expansion.get(index); - } - - @Override - public IntIterator iterator() - { - return new IndexedIntsIterator(this); - } - - @Override - public void fill(int index, int[] toFill) - { - throw new UnsupportedOperationException("fill not supported"); - } - - @Override - public void close() throws IOException - { - - } + public void configure(Binder binder) + { } } diff --git a/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java new file mode 100644 index 000000000000..0423e28d8ade --- /dev/null +++ b/extensions-core/protobuf-extensions/src/main/java/io/druid/data/input/protobuf/ProtobufInputRowParser.java @@ -0,0 +1,151 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.protobuf; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.github.os72.protobuf.dynamic.DynamicSchema; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; +import io.druid.data.input.ByteBufferInputRowParser; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.ParseSpec; +import io.druid.java.util.common.parsers.ParseException; +import io.druid.java.util.common.parsers.Parser; + +import java.io.IOException; +import java.io.InputStream; +import java.net.MalformedURLException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Set; + +public class ProtobufInputRowParser implements ByteBufferInputRowParser +{ + private final ParseSpec parseSpec; + private Parser parser; + private final String descriptorFilePath; + private final String protoMessageType; + private Descriptor descriptor; + + + @JsonCreator + public ProtobufInputRowParser( + @JsonProperty("parseSpec") ParseSpec parseSpec, + @JsonProperty("descriptor") String descriptorFilePath, + @JsonProperty("protoMessageType") String protoMessageType + ) + { + this.parseSpec = parseSpec; + this.descriptorFilePath = descriptorFilePath; + this.protoMessageType = protoMessageType; + this.parser = parseSpec.makeParser(); + this.descriptor = getDescriptor(descriptorFilePath); + } + + @Override + public ParseSpec getParseSpec() + { + return parseSpec; + } + + @Override + public ProtobufInputRowParser withParseSpec(ParseSpec parseSpec) + { + return new ProtobufInputRowParser(parseSpec, descriptorFilePath, protoMessageType); + } + + @Override + public InputRow parse(ByteBuffer input) + { + String json; + try { + DynamicMessage message = DynamicMessage.parseFrom(descriptor, ByteString.copyFrom(input)); + json = JsonFormat.printer().print(message); + } + catch (InvalidProtocolBufferException e) { + throw new ParseException(e, "Protobuf message could not be parsed"); + } + + Map record = parser.parse(json); + return new MapBasedInputRow( + parseSpec.getTimestampSpec().extractTimestamp(record), + parseSpec.getDimensionsSpec().getDimensionNames(), + record + ); + } + + private Descriptor getDescriptor(String descriptorFilePath) + { + InputStream fin; + + fin = this.getClass().getClassLoader().getResourceAsStream(descriptorFilePath); + if (fin == null) { + URL url = null; + try { + url = new URL(descriptorFilePath); + } + catch (MalformedURLException e) { + throw new ParseException(e, "Descriptor not found in class path or malformed URL:" + descriptorFilePath); + } + try { + fin = url.openConnection().getInputStream(); + } + catch (IOException e) { + throw new ParseException(e, "Cannot read descriptor file: " + url.toString()); + } + } + + DynamicSchema dynamicSchema = null; + try { + dynamicSchema = DynamicSchema.parseFrom(fin); + } + catch (Descriptors.DescriptorValidationException e) { + throw new ParseException(e, "Invalid descriptor file: " + descriptorFilePath); + } + catch (IOException e) { + throw new ParseException(e, "Cannot read descriptor file: " + descriptorFilePath); + } + + Set messageTypes = dynamicSchema.getMessageTypes(); + if (messageTypes.size() == 0) { + throw new ParseException("No message types found in the descriptor: " + descriptorFilePath); + } + + String messageType = protoMessageType == null ? (String) messageTypes.toArray()[0] : protoMessageType; + Descriptor desc = dynamicSchema.getMessageDescriptor(messageType); + if (desc == null) { + throw new ParseException( + String.format( + "Protobuf message type %s not found in the specified descriptor. Available messages types are %s", + protoMessageType, + messageTypes + ) + ); + } + return desc; + } +} diff --git a/extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100755 index 000000000000..0a86169f8122 --- /dev/null +++ b/extensions-core/protobuf-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.data.input.protobuf.ProtobufExtensionsModule diff --git a/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java new file mode 100644 index 000000000000..bb16245c5b1f --- /dev/null +++ b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtoTestEventWrapper.java @@ -0,0 +1,3258 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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. + */ + +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: ProtoTest.proto + +package io.druid.data.input.protobuf; + +public final class ProtoTestEventWrapper +{ + private ProtoTestEventWrapper() {} + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistryLite registry + ) + { + } + + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry + ) + { + registerAllExtensions( + (com.google.protobuf.ExtensionRegistryLite) registry); + } + + public interface ProtoTestEventOrBuilder extends + // @@protoc_insertion_point(interface_extends:prototest.ProtoTestEvent) + com.google.protobuf.MessageOrBuilder + { + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + boolean hasEventType(); + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType(); + + /** + * required uint64 id = 2; + */ + boolean hasId(); + + /** + * required uint64 id = 2; + */ + long getId(); + + /** + * required string timestamp = 3; + */ + boolean hasTimestamp(); + + /** + * required string timestamp = 3; + */ + java.lang.String getTimestamp(); + + /** + * required string timestamp = 3; + */ + com.google.protobuf.ByteString + getTimestampBytes(); + + /** + * optional uint32 someOtherId = 4; + */ + boolean hasSomeOtherId(); + + /** + * optional uint32 someOtherId = 4; + */ + int getSomeOtherId(); + + /** + * optional bool isValid = 5; + */ + boolean hasIsValid(); + + /** + * optional bool isValid = 5; + */ + boolean getIsValid(); + + /** + * optional string description = 6; + */ + boolean hasDescription(); + + /** + * optional string description = 6; + */ + java.lang.String getDescription(); + + /** + * optional string description = 6; + */ + com.google.protobuf.ByteString + getDescriptionBytes(); + + /** + * optional float someFloatColumn = 7; + */ + boolean hasSomeFloatColumn(); + + /** + * optional float someFloatColumn = 7; + */ + float getSomeFloatColumn(); + + /** + * optional uint32 someIntColumn = 8; + */ + boolean hasSomeIntColumn(); + + /** + * optional uint32 someIntColumn = 8; + */ + int getSomeIntColumn(); + + /** + * optional uint64 someLongColumn = 9; + */ + boolean hasSomeLongColumn(); + + /** + * optional uint64 someLongColumn = 9; + */ + long getSomeLongColumn(); + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + boolean hasFoo(); + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo(); + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder(); + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + java.util.List + getBarList(); + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index); + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + int getBarCount(); + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + java.util.List + getBarOrBuilderList(); + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder( + int index + ); + } + + /** + * Protobuf type {@code prototest.ProtoTestEvent} + */ + public static final class ProtoTestEvent extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:prototest.ProtoTestEvent) + ProtoTestEventOrBuilder + { + // Use ProtoTestEvent.newBuilder() to construct. + private ProtoTestEvent(com.google.protobuf.GeneratedMessageV3.Builder builder) + { + super(builder); + } + + private ProtoTestEvent() + { + eventType_ = 0; + id_ = 0L; + timestamp_ = ""; + someOtherId_ = 0; + isValid_ = false; + description_ = ""; + someFloatColumn_ = 0F; + someIntColumn_ = 0; + someLongColumn_ = 0L; + bar_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() + { + return this.unknownFields; + } + + private ProtoTestEvent( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + this(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag + )) { + done = true; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory + .valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + eventType_ = rawValue; + } + break; + } + case 16: { + bitField0_ |= 0x00000002; + id_ = input.readUInt64(); + break; + } + case 26: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000004; + timestamp_ = bs; + break; + } + case 32: { + bitField0_ |= 0x00000008; + someOtherId_ = input.readUInt32(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + isValid_ = input.readBool(); + break; + } + case 50: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000020; + description_ = bs; + break; + } + case 61: { + bitField0_ |= 0x00000040; + someFloatColumn_ = input.readFloat(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + someIntColumn_ = input.readUInt32(); + break; + } + case 72: { + bitField0_ |= 0x00000100; + someLongColumn_ = input.readUInt64(); + break; + } + case 82: { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder subBuilder = null; + if (((bitField0_ & 0x00000200) == 0x00000200)) { + subBuilder = foo_.toBuilder(); + } + foo_ = input.readMessage( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER, + extensionRegistry + ); + if (subBuilder != null) { + subBuilder.mergeFrom(foo_); + foo_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000200; + break; + } + case 90: { + if (!((mutable_bitField0_ & 0x00000400) == 0x00000400)) { + bar_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000400; + } + bar_.add( + input.readMessage( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.PARSER, + extensionRegistry + )); + break; + } + } + } + } + catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } + catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } + finally { + if (((mutable_bitField0_ & 0x00000400) == 0x00000400)) { + bar_ = java.util.Collections.unmodifiableList(bar_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class, + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class + ); + } + + /** + * Protobuf enum {@code prototest.ProtoTestEvent.EventCategory} + */ + public enum EventCategory + implements com.google.protobuf.ProtocolMessageEnum + { + /** + * CATEGORY_ZERO = 0; + */ + CATEGORY_ZERO(0), + /** + * CATEGORY_ONE = 1; + */ + CATEGORY_ONE(1), + /** + * CATEGORY_TWO = 2; + */ + CATEGORY_TWO(2),; + + /** + * CATEGORY_ZERO = 0; + */ + public static final int CATEGORY_ZERO_VALUE = 0; + /** + * CATEGORY_ONE = 1; + */ + public static final int CATEGORY_ONE_VALUE = 1; + /** + * CATEGORY_TWO = 2; + */ + public static final int CATEGORY_TWO_VALUE = 2; + + + public final int getNumber() + { + return value; + } + + /** + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static EventCategory valueOf(int value) + { + return forNumber(value); + } + + public static EventCategory forNumber(int value) + { + switch (value) { + case 0: + return CATEGORY_ZERO; + case 1: + return CATEGORY_ONE; + case 2: + return CATEGORY_TWO; + default: + return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() + { + return internalValueMap; + } + + private static final com.google.protobuf.Internal.EnumLiteMap< + EventCategory> internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() + { + public EventCategory findValueByNumber(int number) + { + return EventCategory.forNumber(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() + { + return getDescriptor().getValues().get(ordinal()); + } + + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() + { + return getDescriptor(); + } + + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); + } + + private static final EventCategory[] VALUES = values(); + + public static EventCategory valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc + ) + { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private EventCategory(int value) + { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory) + } + + public interface FooOrBuilder extends + // @@protoc_insertion_point(interface_extends:prototest.ProtoTestEvent.Foo) + com.google.protobuf.MessageOrBuilder + { + + /** + * required string bar = 1; + */ + boolean hasBar(); + + /** + * required string bar = 1; + */ + java.lang.String getBar(); + + /** + * required string bar = 1; + */ + com.google.protobuf.ByteString + getBarBytes(); + } + + /** + * Protobuf type {@code prototest.ProtoTestEvent.Foo} + */ + public static final class Foo extends + com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:prototest.ProtoTestEvent.Foo) + FooOrBuilder + { + // Use Foo.newBuilder() to construct. + private Foo(com.google.protobuf.GeneratedMessageV3.Builder builder) + { + super(builder); + } + + private Foo() + { + bar_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() + { + return this.unknownFields; + } + + private Foo( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + this(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag + )) { + done = true; + } + break; + } + case 10: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + bar_ = bs; + break; + } + } + } + } + catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } + catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e).setUnfinishedMessage(this); + } + finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class, + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class + ); + } + + private int bitField0_; + public static final int BAR_FIELD_NUMBER = 1; + private volatile java.lang.Object bar_; + + /** + * required string bar = 1; + */ + public boolean hasBar() + { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required string bar = 1; + */ + public java.lang.String getBar() + { + java.lang.Object ref = bar_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + bar_ = s; + } + return s; + } + } + + /** + * required string bar = 1; + */ + public com.google.protobuf.ByteString + getBarBytes() + { + java.lang.Object ref = bar_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + bar_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() + { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) { + return true; + } + if (isInitialized == 0) { + return false; + } + + if (!hasBar()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException + { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, bar_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() + { + int size = memoizedSize; + if (size != -1) { + return size; + } + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, bar_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + + @java.lang.Override + public boolean equals(final java.lang.Object obj) + { + if (obj == this) { + return true; + } + if (!(obj instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo)) { + return super.equals(obj); + } + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) obj; + + boolean result = true; + result = result && (hasBar() == other.hasBar()); + if (hasBar()) { + result = result && getBar() + .equals(other.getBar()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() + { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasBar()) { + hash = (37 * hash) + BAR_FIELD_NUMBER; + hash = (53 * hash) + getBar().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom( + com.google.protobuf.ByteString data + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom(java.io.InputStream input) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom( + com.google.protobuf.CodedInputStream input + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + + public static Builder newBuilder() + { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo prototype) + { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() + { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent + ) + { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code prototest.ProtoTestEvent.Foo} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:prototest.ProtoTestEvent.Foo) + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder + { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable + .ensureFieldAccessorsInitialized( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.class, + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder.class + ); + } + + // Construct using io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder() + private Builder() + { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent + ) + { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() + { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + } + } + + public Builder clear() + { + super.clear(); + bar_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_Foo_descriptor; + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance(); + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo build() + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo buildPartial() + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo result = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.bar_ = bar_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() + { + return (Builder) super.clone(); + } + + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + Object value + ) + { + return (Builder) super.setField(field, value); + } + + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field + ) + { + return (Builder) super.clearField(field); + } + + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof + ) + { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value + ) + { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + Object value + ) + { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) + { + if (other instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) { + return mergeFrom((io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo other) + { + if (other == io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) { + return this; + } + if (other.hasBar()) { + bitField0_ |= 0x00000001; + bar_ = other.bar_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() + { + if (!hasBar()) { + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } + catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } + finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object bar_ = ""; + + /** + * required string bar = 1; + */ + public boolean hasBar() + { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required string bar = 1; + */ + public java.lang.String getBar() + { + java.lang.Object ref = bar_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + bar_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * required string bar = 1; + */ + public com.google.protobuf.ByteString + getBarBytes() + { + java.lang.Object ref = bar_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + bar_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * required string bar = 1; + */ + public Builder setBar( + java.lang.String value + ) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + bar_ = value; + onChanged(); + return this; + } + + /** + * required string bar = 1; + */ + public Builder clearBar() + { + bitField0_ = (bitField0_ & ~0x00000001); + bar_ = getDefaultInstance().getBar(); + onChanged(); + return this; + } + + /** + * required string bar = 1; + */ + public Builder setBarBytes( + com.google.protobuf.ByteString value + ) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + bar_ = value; + onChanged(); + return this; + } + + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields + ) + { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields + ) + { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent.Foo) + } + + // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent.Foo) + private static final io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo(); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstance() + { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() + { + public Foo parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return new Foo(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() + { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() + { + return PARSER; + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getDefaultInstanceForType() + { + return DEFAULT_INSTANCE; + } + + } + + private int bitField0_; + public static final int EVENTTYPE_FIELD_NUMBER = 1; + private int eventType_; + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public boolean hasEventType() + { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory + .valueOf(eventType_); + return result == null + ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO + : result; + } + + public static final int ID_FIELD_NUMBER = 2; + private long id_; + + /** + * required uint64 id = 2; + */ + public boolean hasId() + { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * required uint64 id = 2; + */ + public long getId() + { + return id_; + } + + public static final int TIMESTAMP_FIELD_NUMBER = 3; + private volatile java.lang.Object timestamp_; + + /** + * required string timestamp = 3; + */ + public boolean hasTimestamp() + { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * required string timestamp = 3; + */ + public java.lang.String getTimestamp() + { + java.lang.Object ref = timestamp_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + timestamp_ = s; + } + return s; + } + } + + /** + * required string timestamp = 3; + */ + public com.google.protobuf.ByteString + getTimestampBytes() + { + java.lang.Object ref = timestamp_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + timestamp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int SOMEOTHERID_FIELD_NUMBER = 4; + private int someOtherId_; + + /** + * optional uint32 someOtherId = 4; + */ + public boolean hasSomeOtherId() + { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional uint32 someOtherId = 4; + */ + public int getSomeOtherId() + { + return someOtherId_; + } + + public static final int ISVALID_FIELD_NUMBER = 5; + private boolean isValid_; + + /** + * optional bool isValid = 5; + */ + public boolean hasIsValid() + { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + + /** + * optional bool isValid = 5; + */ + public boolean getIsValid() + { + return isValid_; + } + + public static final int DESCRIPTION_FIELD_NUMBER = 6; + private volatile java.lang.Object description_; + + /** + * optional string description = 6; + */ + public boolean hasDescription() + { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + + /** + * optional string description = 6; + */ + public java.lang.String getDescription() + { + java.lang.Object ref = description_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + description_ = s; + } + return s; + } + } + + /** + * optional string description = 6; + */ + public com.google.protobuf.ByteString + getDescriptionBytes() + { + java.lang.Object ref = description_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + description_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7; + private float someFloatColumn_; + + /** + * optional float someFloatColumn = 7; + */ + public boolean hasSomeFloatColumn() + { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + + /** + * optional float someFloatColumn = 7; + */ + public float getSomeFloatColumn() + { + return someFloatColumn_; + } + + public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8; + private int someIntColumn_; + + /** + * optional uint32 someIntColumn = 8; + */ + public boolean hasSomeIntColumn() + { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + + /** + * optional uint32 someIntColumn = 8; + */ + public int getSomeIntColumn() + { + return someIntColumn_; + } + + public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9; + private long someLongColumn_; + + /** + * optional uint64 someLongColumn = 9; + */ + public boolean hasSomeLongColumn() + { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + + /** + * optional uint64 someLongColumn = 9; + */ + public long getSomeLongColumn() + { + return someLongColumn_; + } + + public static final int FOO_FIELD_NUMBER = 10; + private io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_; + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public boolean hasFoo() + { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo() + { + return foo_ == null + ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() + : foo_; + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder() + { + return foo_ == null + ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() + : foo_; + } + + public static final int BAR_FIELD_NUMBER = 11; + private java.util.List bar_; + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public java.util.List getBarList() + { + return bar_; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public java.util.List + getBarOrBuilderList() + { + return bar_; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public int getBarCount() + { + return bar_.size(); + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index) + { + return bar_.get(index); + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder( + int index + ) + { + return bar_.get(index); + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() + { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) { + return true; + } + if (isInitialized == 0) { + return false; + } + + if (!hasEventType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTimestamp()) { + memoizedIsInitialized = 0; + return false; + } + if (hasFoo()) { + if (!getFoo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getBarCount(); i++) { + if (!getBar(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException + { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, eventType_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, id_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, timestamp_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt32(4, someOtherId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(5, isValid_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 6, description_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeFloat(7, someFloatColumn_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeUInt32(8, someIntColumn_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeUInt64(9, someLongColumn_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeMessage(10, getFoo()); + } + for (int i = 0; i < bar_.size(); i++) { + output.writeMessage(11, bar_.get(i)); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() + { + int size = memoizedSize; + if (size != -1) { + return size; + } + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, eventType_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, id_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, timestamp_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(4, someOtherId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(5, isValid_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(6, description_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize(7, someFloatColumn_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(8, someIntColumn_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(9, someLongColumn_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(10, getFoo()); + } + for (int i = 0; i < bar_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(11, bar_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + + @java.lang.Override + public boolean equals(final java.lang.Object obj) + { + if (obj == this) { + return true; + } + if (!(obj instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent)) { + return super.equals(obj); + } + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) obj; + + boolean result = true; + result = result && (hasEventType() == other.hasEventType()); + if (hasEventType()) { + result = result && eventType_ == other.eventType_; + } + result = result && (hasId() == other.hasId()); + if (hasId()) { + result = result && (getId() + == other.getId()); + } + result = result && (hasTimestamp() == other.hasTimestamp()); + if (hasTimestamp()) { + result = result && getTimestamp() + .equals(other.getTimestamp()); + } + result = result && (hasSomeOtherId() == other.hasSomeOtherId()); + if (hasSomeOtherId()) { + result = result && (getSomeOtherId() + == other.getSomeOtherId()); + } + result = result && (hasIsValid() == other.hasIsValid()); + if (hasIsValid()) { + result = result && (getIsValid() + == other.getIsValid()); + } + result = result && (hasDescription() == other.hasDescription()); + if (hasDescription()) { + result = result && getDescription() + .equals(other.getDescription()); + } + result = result && (hasSomeFloatColumn() == other.hasSomeFloatColumn()); + if (hasSomeFloatColumn()) { + result = result && ( + java.lang.Float.floatToIntBits(getSomeFloatColumn()) + == java.lang.Float.floatToIntBits( + other.getSomeFloatColumn())); + } + result = result && (hasSomeIntColumn() == other.hasSomeIntColumn()); + if (hasSomeIntColumn()) { + result = result && (getSomeIntColumn() + == other.getSomeIntColumn()); + } + result = result && (hasSomeLongColumn() == other.hasSomeLongColumn()); + if (hasSomeLongColumn()) { + result = result && (getSomeLongColumn() + == other.getSomeLongColumn()); + } + result = result && (hasFoo() == other.hasFoo()); + if (hasFoo()) { + result = result && getFoo() + .equals(other.getFoo()); + } + result = result && getBarList() + .equals(other.getBarList()); + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() + { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasEventType()) { + hash = (37 * hash) + EVENTTYPE_FIELD_NUMBER; + hash = (53 * hash) + eventType_; + } + if (hasId()) { + hash = (37 * hash) + ID_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getId()); + } + if (hasTimestamp()) { + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + getTimestamp().hashCode(); + } + if (hasSomeOtherId()) { + hash = (37 * hash) + SOMEOTHERID_FIELD_NUMBER; + hash = (53 * hash) + getSomeOtherId(); + } + if (hasIsValid()) { + hash = (37 * hash) + ISVALID_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean( + getIsValid()); + } + if (hasDescription()) { + hash = (37 * hash) + DESCRIPTION_FIELD_NUMBER; + hash = (53 * hash) + getDescription().hashCode(); + } + if (hasSomeFloatColumn()) { + hash = (37 * hash) + SOMEFLOATCOLUMN_FIELD_NUMBER; + hash = (53 * hash) + java.lang.Float.floatToIntBits( + getSomeFloatColumn()); + } + if (hasSomeIntColumn()) { + hash = (37 * hash) + SOMEINTCOLUMN_FIELD_NUMBER; + hash = (53 * hash) + getSomeIntColumn(); + } + if (hasSomeLongColumn()) { + hash = (37 * hash) + SOMELONGCOLUMN_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getSomeLongColumn()); + } + if (hasFoo()) { + hash = (37 * hash) + FOO_FIELD_NUMBER; + hash = (53 * hash) + getFoo().hashCode(); + } + if (getBarCount() > 0) { + hash = (37 * hash) + BAR_FIELD_NUMBER; + hash = (53 * hash) + getBarList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.ByteString data + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseDelimitedWithIOException(PARSER, input, extensionRegistry); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.CodedInputStream input + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + return com.google.protobuf.GeneratedMessageV3 + .parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { return newBuilder(); } + + public static Builder newBuilder() + { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent prototype) + { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() + { + return this == DEFAULT_INSTANCE + ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent + ) + { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code prototest.ProtoTestEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessageV3.Builder implements + // @@protoc_insertion_point(builder_implements:prototest.ProtoTestEvent) + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEventOrBuilder + { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.class, + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Builder.class + ); + } + + // Construct using io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + private Builder() + { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessageV3.BuilderParent parent + ) + { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() + { + if (com.google.protobuf.GeneratedMessageV3 + .alwaysUseFieldBuilders) { + getFooFieldBuilder(); + getBarFieldBuilder(); + } + } + + public Builder clear() + { + super.clear(); + eventType_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + id_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + someOtherId_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + isValid_ = false; + bitField0_ = (bitField0_ & ~0x00000010); + description_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + someFloatColumn_ = 0F; + bitField0_ = (bitField0_ & ~0x00000040); + someIntColumn_ = 0; + bitField0_ = (bitField0_ & ~0x00000080); + someLongColumn_ = 0L; + bitField0_ = (bitField0_ & ~0x00000100); + if (fooBuilder_ == null) { + foo_ = null; + } else { + fooBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000200); + if (barBuilder_ == null) { + bar_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000400); + } else { + barBuilder_.clear(); + } + return this; + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() + { + return io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent build() + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent buildPartial() + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent result = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.eventType_ = eventType_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.id_ = id_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.timestamp_ = timestamp_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.someOtherId_ = someOtherId_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.isValid_ = isValid_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.description_ = description_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.someFloatColumn_ = someFloatColumn_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.someIntColumn_ = someIntColumn_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.someLongColumn_ = someLongColumn_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + if (fooBuilder_ == null) { + result.foo_ = foo_; + } else { + result.foo_ = fooBuilder_.build(); + } + if (barBuilder_ == null) { + if (((bitField0_ & 0x00000400) == 0x00000400)) { + bar_ = java.util.Collections.unmodifiableList(bar_); + bitField0_ = (bitField0_ & ~0x00000400); + } + result.bar_ = bar_; + } else { + result.bar_ = barBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() + { + return (Builder) super.clone(); + } + + public Builder setField( + com.google.protobuf.Descriptors.FieldDescriptor field, + Object value + ) + { + return (Builder) super.setField(field, value); + } + + public Builder clearField( + com.google.protobuf.Descriptors.FieldDescriptor field + ) + { + return (Builder) super.clearField(field); + } + + public Builder clearOneof( + com.google.protobuf.Descriptors.OneofDescriptor oneof + ) + { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + int index, Object value + ) + { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField( + com.google.protobuf.Descriptors.FieldDescriptor field, + Object value + ) + { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) + { + if (other instanceof io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) { + return mergeFrom((io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent other) + { + if (other == io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) { + return this; + } + if (other.hasEventType()) { + setEventType(other.getEventType()); + } + if (other.hasId()) { + setId(other.getId()); + } + if (other.hasTimestamp()) { + bitField0_ |= 0x00000004; + timestamp_ = other.timestamp_; + onChanged(); + } + if (other.hasSomeOtherId()) { + setSomeOtherId(other.getSomeOtherId()); + } + if (other.hasIsValid()) { + setIsValid(other.getIsValid()); + } + if (other.hasDescription()) { + bitField0_ |= 0x00000020; + description_ = other.description_; + onChanged(); + } + if (other.hasSomeFloatColumn()) { + setSomeFloatColumn(other.getSomeFloatColumn()); + } + if (other.hasSomeIntColumn()) { + setSomeIntColumn(other.getSomeIntColumn()); + } + if (other.hasSomeLongColumn()) { + setSomeLongColumn(other.getSomeLongColumn()); + } + if (other.hasFoo()) { + mergeFoo(other.getFoo()); + } + if (barBuilder_ == null) { + if (!other.bar_.isEmpty()) { + if (bar_.isEmpty()) { + bar_ = other.bar_; + bitField0_ = (bitField0_ & ~0x00000400); + } else { + ensureBarIsMutable(); + bar_.addAll(other.bar_); + } + onChanged(); + } + } else { + if (!other.bar_.isEmpty()) { + if (barBuilder_.isEmpty()) { + barBuilder_.dispose(); + barBuilder_ = null; + bar_ = other.bar_; + bitField0_ = (bitField0_ & ~0x00000400); + barBuilder_ = + com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ? + getBarFieldBuilder() : null; + } else { + barBuilder_.addAllMessages(other.bar_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() + { + if (!hasEventType()) { + return false; + } + if (!hasId()) { + return false; + } + if (!hasTimestamp()) { + return false; + } + if (hasFoo()) { + if (!getFoo().isInitialized()) { + return false; + } + } + for (int i = 0; i < getBarCount(); i++) { + if (!getBar(i).isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws java.io.IOException + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } + catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } + finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private int eventType_ = 0; + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public boolean hasEventType() + { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() + { + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory result = io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory + .valueOf(eventType_); + return result == null + ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO + : result; + } + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public Builder setEventType(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + eventType_ = value.getNumber(); + onChanged(); + return this; + } + + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public Builder clearEventType() + { + bitField0_ = (bitField0_ & ~0x00000001); + eventType_ = 0; + onChanged(); + return this; + } + + private long id_; + + /** + * required uint64 id = 2; + */ + public boolean hasId() + { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * required uint64 id = 2; + */ + public long getId() + { + return id_; + } + + /** + * required uint64 id = 2; + */ + public Builder setId(long value) + { + bitField0_ |= 0x00000002; + id_ = value; + onChanged(); + return this; + } + + /** + * required uint64 id = 2; + */ + public Builder clearId() + { + bitField0_ = (bitField0_ & ~0x00000002); + id_ = 0L; + onChanged(); + return this; + } + + private java.lang.Object timestamp_ = ""; + + /** + * required string timestamp = 3; + */ + public boolean hasTimestamp() + { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * required string timestamp = 3; + */ + public java.lang.String getTimestamp() + { + java.lang.Object ref = timestamp_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + timestamp_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * required string timestamp = 3; + */ + public com.google.protobuf.ByteString + getTimestampBytes() + { + java.lang.Object ref = timestamp_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + timestamp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * required string timestamp = 3; + */ + public Builder setTimestamp( + java.lang.String value + ) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + timestamp_ = value; + onChanged(); + return this; + } + + /** + * required string timestamp = 3; + */ + public Builder clearTimestamp() + { + bitField0_ = (bitField0_ & ~0x00000004); + timestamp_ = getDefaultInstance().getTimestamp(); + onChanged(); + return this; + } + + /** + * required string timestamp = 3; + */ + public Builder setTimestampBytes( + com.google.protobuf.ByteString value + ) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + timestamp_ = value; + onChanged(); + return this; + } + + private int someOtherId_; + + /** + * optional uint32 someOtherId = 4; + */ + public boolean hasSomeOtherId() + { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional uint32 someOtherId = 4; + */ + public int getSomeOtherId() + { + return someOtherId_; + } + + /** + * optional uint32 someOtherId = 4; + */ + public Builder setSomeOtherId(int value) + { + bitField0_ |= 0x00000008; + someOtherId_ = value; + onChanged(); + return this; + } + + /** + * optional uint32 someOtherId = 4; + */ + public Builder clearSomeOtherId() + { + bitField0_ = (bitField0_ & ~0x00000008); + someOtherId_ = 0; + onChanged(); + return this; + } + + private boolean isValid_; + + /** + * optional bool isValid = 5; + */ + public boolean hasIsValid() + { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + + /** + * optional bool isValid = 5; + */ + public boolean getIsValid() + { + return isValid_; + } + + /** + * optional bool isValid = 5; + */ + public Builder setIsValid(boolean value) + { + bitField0_ |= 0x00000010; + isValid_ = value; + onChanged(); + return this; + } + + /** + * optional bool isValid = 5; + */ + public Builder clearIsValid() + { + bitField0_ = (bitField0_ & ~0x00000010); + isValid_ = false; + onChanged(); + return this; + } + + private java.lang.Object description_ = ""; + + /** + * optional string description = 6; + */ + public boolean hasDescription() + { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + + /** + * optional string description = 6; + */ + public java.lang.String getDescription() + { + java.lang.Object ref = description_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + description_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * optional string description = 6; + */ + public com.google.protobuf.ByteString + getDescriptionBytes() + { + java.lang.Object ref = description_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + description_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * optional string description = 6; + */ + public Builder setDescription( + java.lang.String value + ) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + description_ = value; + onChanged(); + return this; + } + + /** + * optional string description = 6; + */ + public Builder clearDescription() + { + bitField0_ = (bitField0_ & ~0x00000020); + description_ = getDefaultInstance().getDescription(); + onChanged(); + return this; + } + + /** + * optional string description = 6; + */ + public Builder setDescriptionBytes( + com.google.protobuf.ByteString value + ) + { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + description_ = value; + onChanged(); + return this; + } + + private float someFloatColumn_; + + /** + * optional float someFloatColumn = 7; + */ + public boolean hasSomeFloatColumn() + { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + + /** + * optional float someFloatColumn = 7; + */ + public float getSomeFloatColumn() + { + return someFloatColumn_; + } + + /** + * optional float someFloatColumn = 7; + */ + public Builder setSomeFloatColumn(float value) + { + bitField0_ |= 0x00000040; + someFloatColumn_ = value; + onChanged(); + return this; + } + + /** + * optional float someFloatColumn = 7; + */ + public Builder clearSomeFloatColumn() + { + bitField0_ = (bitField0_ & ~0x00000040); + someFloatColumn_ = 0F; + onChanged(); + return this; + } + + private int someIntColumn_; + + /** + * optional uint32 someIntColumn = 8; + */ + public boolean hasSomeIntColumn() + { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + + /** + * optional uint32 someIntColumn = 8; + */ + public int getSomeIntColumn() + { + return someIntColumn_; + } + + /** + * optional uint32 someIntColumn = 8; + */ + public Builder setSomeIntColumn(int value) + { + bitField0_ |= 0x00000080; + someIntColumn_ = value; + onChanged(); + return this; + } + + /** + * optional uint32 someIntColumn = 8; + */ + public Builder clearSomeIntColumn() + { + bitField0_ = (bitField0_ & ~0x00000080); + someIntColumn_ = 0; + onChanged(); + return this; + } + + private long someLongColumn_; + + /** + * optional uint64 someLongColumn = 9; + */ + public boolean hasSomeLongColumn() + { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + + /** + * optional uint64 someLongColumn = 9; + */ + public long getSomeLongColumn() + { + return someLongColumn_; + } + + /** + * optional uint64 someLongColumn = 9; + */ + public Builder setSomeLongColumn(long value) + { + bitField0_ |= 0x00000100; + someLongColumn_ = value; + onChanged(); + return this; + } + + /** + * optional uint64 someLongColumn = 9; + */ + public Builder clearSomeLongColumn() + { + bitField0_ = (bitField0_ & ~0x00000100); + someLongColumn_ = 0L; + onChanged(); + return this; + } + + private io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo foo_ = null; + private com.google.protobuf.SingleFieldBuilderV3< + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> fooBuilder_; + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public boolean hasFoo() + { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getFoo() + { + if (fooBuilder_ == null) { + return foo_ == null + ? io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() + : foo_; + } else { + return fooBuilder_.getMessage(); + } + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public Builder setFoo(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) + { + if (fooBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + foo_ = value; + onChanged(); + } else { + fooBuilder_.setMessage(value); + } + bitField0_ |= 0x00000200; + return this; + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public Builder setFoo( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue + ) + { + if (fooBuilder_ == null) { + foo_ = builderForValue.build(); + onChanged(); + } else { + fooBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000200; + return this; + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public Builder mergeFoo(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) + { + if (fooBuilder_ == null) { + if (((bitField0_ & 0x00000200) == 0x00000200) && + foo_ != null && + foo_ != io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()) { + foo_ = + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.newBuilder(foo_) + .mergeFrom(value) + .buildPartial(); + } else { + foo_ = value; + } + onChanged(); + } else { + fooBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000200; + return this; + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public Builder clearFoo() + { + if (fooBuilder_ == null) { + foo_ = null; + onChanged(); + } else { + fooBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000200); + return this; + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getFooBuilder() + { + bitField0_ |= 0x00000200; + onChanged(); + return getFooFieldBuilder().getBuilder(); + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getFooOrBuilder() + { + if (fooBuilder_ != null) { + return fooBuilder_.getMessageOrBuilder(); + } else { + return foo_ == null ? + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance() : foo_; + } + } + + /** + * optional .prototest.ProtoTestEvent.Foo foo = 10; + */ + private com.google.protobuf.SingleFieldBuilderV3< + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> + getFooFieldBuilder() + { + if (fooBuilder_ == null) { + fooBuilder_ = new com.google.protobuf.SingleFieldBuilderV3< + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>( + getFoo(), + getParentForChildren(), + isClean() + ); + foo_ = null; + } + return fooBuilder_; + } + + private java.util.List bar_ = + java.util.Collections.emptyList(); + + private void ensureBarIsMutable() + { + if (!((bitField0_ & 0x00000400) == 0x00000400)) { + bar_ = new java.util.ArrayList(bar_); + bitField0_ |= 0x00000400; + } + } + + private com.google.protobuf.RepeatedFieldBuilderV3< + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> barBuilder_; + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public java.util.List getBarList() + { + if (barBuilder_ == null) { + return java.util.Collections.unmodifiableList(bar_); + } else { + return barBuilder_.getMessageList(); + } + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public int getBarCount() + { + if (barBuilder_ == null) { + return bar_.size(); + } else { + return barBuilder_.getCount(); + } + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo getBar(int index) + { + if (barBuilder_ == null) { + return bar_.get(index); + } else { + return barBuilder_.getMessage(index); + } + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder setBar( + int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value + ) + { + if (barBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureBarIsMutable(); + bar_.set(index, value); + onChanged(); + } else { + barBuilder_.setMessage(index, value); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder setBar( + int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue + ) + { + if (barBuilder_ == null) { + ensureBarIsMutable(); + bar_.set(index, builderForValue.build()); + onChanged(); + } else { + barBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder addBar(io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value) + { + if (barBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureBarIsMutable(); + bar_.add(value); + onChanged(); + } else { + barBuilder_.addMessage(value); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder addBar( + int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo value + ) + { + if (barBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureBarIsMutable(); + bar_.add(index, value); + onChanged(); + } else { + barBuilder_.addMessage(index, value); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder addBar( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue + ) + { + if (barBuilder_ == null) { + ensureBarIsMutable(); + bar_.add(builderForValue.build()); + onChanged(); + } else { + barBuilder_.addMessage(builderForValue.build()); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder addBar( + int index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder builderForValue + ) + { + if (barBuilder_ == null) { + ensureBarIsMutable(); + bar_.add(index, builderForValue.build()); + onChanged(); + } else { + barBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder addAllBar( + java.lang.Iterable values + ) + { + if (barBuilder_ == null) { + ensureBarIsMutable(); + com.google.protobuf.AbstractMessageLite.Builder.addAll( + values, bar_); + onChanged(); + } else { + barBuilder_.addAllMessages(values); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder clearBar() + { + if (barBuilder_ == null) { + bar_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000400); + onChanged(); + } else { + barBuilder_.clear(); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public Builder removeBar(int index) + { + if (barBuilder_ == null) { + ensureBarIsMutable(); + bar_.remove(index); + onChanged(); + } else { + barBuilder_.remove(index); + } + return this; + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder getBarBuilder( + int index + ) + { + return getBarFieldBuilder().getBuilder(index); + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder getBarOrBuilder( + int index + ) + { + if (barBuilder_ == null) { + return bar_.get(index); + } else { + return barBuilder_.getMessageOrBuilder(index); + } + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public java.util.List + getBarOrBuilderList() + { + if (barBuilder_ != null) { + return barBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(bar_); + } + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder() + { + return getBarFieldBuilder().addBuilder( + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()); + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder addBarBuilder( + int index + ) + { + return getBarFieldBuilder().addBuilder( + index, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.getDefaultInstance()); + } + + /** + * repeated .prototest.ProtoTestEvent.Foo bar = 11; + */ + public java.util.List + getBarBuilderList() + { + return getBarFieldBuilder().getBuilderList(); + } + + private com.google.protobuf.RepeatedFieldBuilderV3< + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder> + getBarFieldBuilder() + { + if (barBuilder_ == null) { + barBuilder_ = new com.google.protobuf.RepeatedFieldBuilderV3< + io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.Foo.Builder, io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent.FooOrBuilder>( + bar_, + ((bitField0_ & 0x00000400) == 0x00000400), + getParentForChildren(), + isClean() + ); + bar_ = null; + } + return barBuilder_; + } + + public final Builder setUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields + ) + { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields( + final com.google.protobuf.UnknownFieldSet unknownFields + ) + { + return super.mergeUnknownFields(unknownFields); + } + + + // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent) + } + + // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent) + private static final io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent(); + } + + public static io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstance() + { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser + PARSER = new com.google.protobuf.AbstractParser() + { + public ProtoTestEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry + ) + throws com.google.protobuf.InvalidProtocolBufferException + { + return new ProtoTestEvent(input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() + { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() + { + return PARSER; + } + + public io.druid.data.input.protobuf.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() + { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_prototest_ProtoTestEvent_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_prototest_ProtoTestEvent_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor + internal_static_prototest_ProtoTestEvent_Foo_descriptor; + private static final + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable + internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() + { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + + static { + java.lang.String[] descriptorData = { + "\n\017ProtoTest.proto\022\tprototest\"\242\003\n\016ProtoTe" + + "stEvent\022:\n\teventType\030\001 \002(\0162\'.prototest.P" + + "rotoTestEvent.EventCategory\022\n\n\002id\030\002 \002(\004\022" + + "\021\n\ttimestamp\030\003 \002(\t\022\023\n\013someOtherId\030\004 \001(\r\022" + + "\017\n\007isValid\030\005 \001(\010\022\023\n\013description\030\006 \001(\t\022\027\n" + + "\017someFloatColumn\030\007 \001(\002\022\025\n\rsomeIntColumn\030" + + "\010 \001(\r\022\026\n\016someLongColumn\030\t \001(\004\022*\n\003foo\030\n \001" + + "(\0132\035.prototest.ProtoTestEvent.Foo\022*\n\003bar" + + "\030\013 \003(\0132\035.prototest.ProtoTestEvent.Foo\032\022\n" + + "\003Foo\022\013\n\003bar\030\001 \002(\t\"F\n\rEventCategory\022\021\n\rCA", + "TEGORY_ZERO\020\000\022\020\n\014CATEGORY_ONE\020\001\022\020\n\014CATEG" + + "ORY_TWO\020\002B5\n\034io.druid.data.input.protobu" + + "fB\025ProtoTestEventWrapper" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() + { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root + ) + { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[]{ + }, assigner + ); + internal_static_prototest_ProtoTestEvent_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_prototest_ProtoTestEvent_descriptor, + new java.lang.String[]{ + "EventType", + "Id", + "Timestamp", + "SomeOtherId", + "IsValid", + "Description", + "SomeFloatColumn", + "SomeIntColumn", + "SomeLongColumn", + "Foo", + "Bar", + } + ); + internal_static_prototest_ProtoTestEvent_Foo_descriptor = + internal_static_prototest_ProtoTestEvent_descriptor.getNestedTypes().get(0); + internal_static_prototest_ProtoTestEvent_Foo_fieldAccessorTable = new + com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_prototest_ProtoTestEvent_Foo_descriptor, + new java.lang.String[]{"Bar",} + ); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java new file mode 100644 index 000000000000..19a95087ca1d --- /dev/null +++ b/extensions-core/protobuf-extensions/src/test/java/io/druid/data/input/protobuf/ProtobufInputRowParserTest.java @@ -0,0 +1,170 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.protobuf; + +import com.google.common.collect.Lists; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.JSONPathFieldSpec; +import io.druid.data.input.impl.JSONPathFieldType; +import io.druid.data.input.impl.JSONPathSpec; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.parsers.ParseException; +import org.joda.time.DateTime; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.nio.ByteBuffer; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class ProtobufInputRowParserTest +{ + private ParseSpec parseSpec; + + @Before + public void setUp() throws Exception + { + parseSpec = new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(Lists.newArrayList( + new StringDimensionSchema("event"), + new StringDimensionSchema("id"), + new StringDimensionSchema("someOtherId"), + new StringDimensionSchema("isValid") + ), null, null), + new JSONPathSpec( + true, + Lists.newArrayList( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "eventType", "eventType"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "foobar", "$.foo.bar"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "bar0", "$.bar[0].bar") + ) + ), null + ); + + } + + @Test + public void testShortMessageType() throws Exception + { + //configure parser with desc file, and specify which file name to use + ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent"); + + } + + + @Test + public void testLongMessageType() throws Exception + { + //configure parser with desc file, and specify which file name to use + ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "prototest.ProtoTestEvent"); + + } + + + @Test(expected = ParseException.class) + public void testBadProto() throws Exception + { + //configure parser with desc file + ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "BadName"); + + } + + @Test(expected = ParseException.class) + public void testMalformedDescriptorUrl() throws Exception + { + //configure parser with non existent desc file + ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "file:/nonexist.desc", "BadName"); + } + + @Test + public void testSingleDescriptorNoMessageType() throws Exception + { + // For the backward compatibility, protoMessageType allows null when the desc file has only one message type. + ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", null); + } + + @Test + public void testParse() throws Exception + { + + //configure parser with desc file + ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent"); + + //create binary of proto test event + DateTime dateTime = new DateTime(2012, 07, 12, 9, 30); + ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + .setDescription("description") + .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) + .setId(4711L) + .setIsValid(true) + .setSomeOtherId(4712) + .setTimestamp(dateTime.toString()) + .setSomeFloatColumn(47.11F) + .setSomeIntColumn(815) + .setSomeLongColumn(816L) + .setFoo(ProtoTestEventWrapper.ProtoTestEvent.Foo + .newBuilder() + .setBar("baz")) + .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo + .newBuilder() + .setBar("bar0")) + .addBar(ProtoTestEventWrapper.ProtoTestEvent.Foo + .newBuilder() + .setBar("bar1")) + .build(); + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + event.writeTo(out); + + InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray())); + System.out.println(row); + + assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); + + assertDimensionEquals(row, "id", "4711"); + assertDimensionEquals(row, "isValid", "true"); + assertDimensionEquals(row, "someOtherId", "4712"); + assertDimensionEquals(row, "description", "description"); + + assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name()); + assertDimensionEquals(row, "foobar", "baz"); + assertDimensionEquals(row, "bar0", "bar0"); + + + assertEquals(47.11F, row.getFloatMetric("someFloatColumn"), 0.0); + assertEquals(815.0F, row.getFloatMetric("someIntColumn"), 0.0); + assertEquals(816.0F, row.getFloatMetric("someLongColumn"), 0.0); + } + + private void assertDimensionEquals(InputRow row, String dimension, Object expected) + { + List values = row.getDimension(dimension); + assertEquals(1, values.size()); + assertEquals(expected, values.get(0)); + } +} diff --git a/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto b/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto new file mode 100644 index 000000000000..1a68ab09fe72 --- /dev/null +++ b/extensions-core/protobuf-extensions/src/test/resources/ProtoTest.proto @@ -0,0 +1,30 @@ +syntax = "proto2"; +package prototest; +option java_package = "io.druid.data.input.protobuf"; +option java_outer_classname = "ProtoTestEventWrapper"; + + +message ProtoTestEvent { + enum EventCategory { + CATEGORY_ZERO = 0; + CATEGORY_ONE = 1; + CATEGORY_TWO = 2; + } + + message Foo { + required string bar = 1; + } + + required EventCategory eventType = 1; + required uint64 id = 2; + required string timestamp = 3; + optional uint32 someOtherId = 4; + optional bool isValid = 5; + optional string description = 6; + + optional float someFloatColumn = 7; + optional uint32 someIntColumn = 8; + optional uint64 someLongColumn = 9; + optional Foo foo = 10; + repeated Foo bar = 11; +} diff --git a/extensions-core/protobuf-extensions/src/test/resources/prototest.desc b/extensions-core/protobuf-extensions/src/test/resources/prototest.desc new file mode 100644 index 0000000000000000000000000000000000000000..af7d67d102ca891b0bec0c20c3661e373db2ecd1 GIT binary patch literal 631 zcma))?MlNi6oy?}-Fjxb8W4j_C_{9=Dh#{;H#!-Jl~5e|M>?C$!rGKH6?QXTr59kb z54R2kfA#b^@AFccz_$UdC(Z;5X(7B^O~_n6gan;01MJ_m|4B1}d@!jR!bMJT&DFYg z?|LOF!>gvfr89^GoiV;3TNBx!B{9~dZxA`5HkcwwWkZzaSeMu&TYy|>7A&QK_@I0g zW5ca=EwYOchI&#wM{`+vD7Ou=c~x#qi+7&nB4HUml3SDPBGiSJY9G#7BnE6=rWvlw zlt%V3x&%ulN7*LVSbj+rqARdf>Vajmb)6|=2HB(N2J|Uo8050qyZHMj4jCgVsf03r>XX1Zkk;`p3Lbr@Rw3ay laJ2p{v=ipVyi8&*jzr`oSzfL$(xjaBPpkJ~%%eP~{0A0tzkUD! literal 0 HcmV?d00001 diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index b2ccd54d73f1..95ae9394a70a 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -22,54 +22,67 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; -import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.logger.Logger; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; +import org.jets3t.service.ServiceException; +import org.jets3t.service.StorageObjectsChunk; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; -import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.net.URI; -import java.util.Iterator; -import java.util.LinkedList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.stream.Collectors; /** * Builds firehoses that read from a predefined list of S3 objects and then dry up. */ -public class StaticS3FirehoseFactory implements FirehoseFactory +public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); + private static final long MAX_LISTING_LENGTH = 1024; private final RestS3Service s3Client; private final List uris; + private final List prefixes; @JsonCreator public StaticS3FirehoseFactory( @JacksonInject("s3Client") RestS3Service s3Client, - @JsonProperty("uris") List uris + @JsonProperty("uris") List uris, + @JsonProperty("prefixes") List prefixes, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Long fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry ) { - this.s3Client = s3Client; - this.uris = ImmutableList.copyOf(uris); + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + this.s3Client = Preconditions.checkNotNull(s3Client, "null s3Client"); + this.uris = uris == null ? new ArrayList<>() : uris; + this.prefixes = prefixes == null ? new ArrayList<>() : prefixes; + + if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) { + throw new IAE("uris and directories cannot be used together"); + } - for (final URI inputURI : uris) { + if (this.uris.isEmpty() && this.prefixes.isEmpty()) { + throw new IAE("uris or directories must be specified"); + } + + for (final URI inputURI : this.uris) { + Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); + } + + for (final URI inputURI : this.prefixes) { Preconditions.checkArgument(inputURI.getScheme().equals("s3"), "input uri scheme == s3 (%s)", inputURI); } } @@ -80,72 +93,81 @@ public List getUris() return uris; } + @JsonProperty("prefixes") + public List getPrefixes() + { + return prefixes; + } + @Override - public Firehose connect(StringInputRowParser firehoseParser) throws IOException + protected Collection initObjects() throws IOException { - Preconditions.checkNotNull(s3Client, "null s3Client"); - - final LinkedList objectQueue = Lists.newLinkedList(uris); - - return new FileIteratingFirehose( - new Iterator() - { - @Override - public boolean hasNext() - { - return !objectQueue.isEmpty(); - } - - @Override - public LineIterator next() - { - final URI nextURI = objectQueue.poll(); - - final String s3Bucket = nextURI.getAuthority(); - final S3Object s3Object = new S3Object( - nextURI.getPath().startsWith("/") - ? nextURI.getPath().substring(1) - : nextURI.getPath() + // Here, the returned s3 objects contain minimal information without data. + // Getting data is deferred until openObjectStream() is called for each object. + if (!uris.isEmpty()) { + return uris.stream() + .map( + uri -> { + final String s3Bucket = uri.getAuthority(); + final S3Object s3Object = new S3Object(extractS3Key(uri)); + s3Object.setBucketName(s3Bucket); + return s3Object; + } + ) + .collect(Collectors.toList()); + } else { + final List objects = new ArrayList<>(); + for (URI uri : prefixes) { + final String bucket = uri.getAuthority(); + final String prefix = extractS3Key(uri); + try { + String lastKey = null; + StorageObjectsChunk objectsChunk; + do { + objectsChunk = s3Client.listObjectsChunked( + bucket, + prefix, + null, + MAX_LISTING_LENGTH, + lastKey ); + Arrays.stream(objectsChunk.getObjects()).forEach(storageObject -> objects.add((S3Object) storageObject)); + lastKey = objectsChunk.getPriorLastKey(); + } while (!objectsChunk.isListingComplete()); + } + catch (ServiceException e) { + throw new IOException(e); + } + } + return objects; + } + } - log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), nextURI); - - try { - final InputStream innerInputStream = s3Client.getObject( - new S3Bucket(s3Bucket), s3Object.getKey() - ) - .getDataInputStream(); - - final InputStream outerInputStream = s3Object.getKey().endsWith(".gz") - ? CompressionUtils.gzipInputStream(innerInputStream) - : innerInputStream; - - return IOUtils.lineIterator( - new BufferedReader( - new InputStreamReader(outerInputStream, Charsets.UTF_8) - ) - ); - } - catch (Exception e) { - log.error( - e, - "Exception reading from bucket[%s] object[%s]", - s3Bucket, - s3Object.getKey() - ); - - throw Throwables.propagate(e); - } - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }, - firehoseParser - ); + private static String extractS3Key(URI uri) + { + return uri.getPath().startsWith("/") + ? uri.getPath().substring(1) + : uri.getPath(); + } + + @Override + protected InputStream openObjectStream(S3Object object) throws IOException + { + log.info("Reading from bucket[%s] object[%s] (%s)", object.getBucketName(), object.getKey(), object); + + try { + // Get data of the given object and open an input stream + return s3Client.getObject(object.getBucketName(), object.getKey()).getDataInputStream(); + } + catch (ServiceException e) { + throw new IOException(e); + } + } + + @Override + protected InputStream wrapObjectStream(S3Object object, InputStream stream) throws IOException + { + return object.getKey().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } @Override @@ -160,13 +182,13 @@ public boolean equals(Object o) StaticS3FirehoseFactory factory = (StaticS3FirehoseFactory) o; - return !(uris != null ? !uris.equals(factory.uris) : factory.uris != null); + return getUris().equals(factory.getUris()); } @Override public int hashCode() { - return uris != null ? uris.hashCode() : 0; + return getUris().hashCode(); } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java index d5bc25677fc9..abaa8f030763 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java @@ -54,6 +54,7 @@ public String getSecretKey() { return provider.getCredentials().getAWSSecretKey(); } + @Override public String getSessionToken() { com.amazonaws.auth.AWSSessionCredentials sessionCredentials = (com.amazonaws.auth.AWSSessionCredentials) provider.getCredentials(); diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 352d6cca15ac..23b331f63251 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -30,6 +30,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.segment.loading.DataSegmentMover; +import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; @@ -69,7 +70,7 @@ public DataSegment move(DataSegment segment, Map targetLoadSpec) final String targetS3Bucket = MapUtils.getString(targetLoadSpec, "bucket"); final String targetS3BaseKey = MapUtils.getString(targetLoadSpec, "baseKey"); - final String targetS3Path = S3Utils.constructSegmentPath(targetS3BaseKey, segment); + final String targetS3Path = S3Utils.constructSegmentPath(targetS3BaseKey, DataSegmentPusher.getDefaultStorageDir(segment)); String targetS3DescriptorPath = S3Utils.descriptorPathForSegmentPath(targetS3Path); if (targetS3Bucket.isEmpty()) { diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java index eb7ca2ddd2bd..56b121452542 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java @@ -363,6 +363,7 @@ public S3Coords(String bucket, String key) this.path = key; } + @Override public String toString() { return String.format("s3://%s/%s", bucket, path); diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index a035ff20c22f..8f9e99256df3 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -38,6 +38,8 @@ import java.io.File; import java.io.IOException; +import java.net.URI; +import java.util.Map; import java.util.concurrent.Callable; public class S3DataSegmentPusher implements DataSegmentPusher @@ -65,6 +67,9 @@ public S3DataSegmentPusher( @Override public String getPathForHadoop() { + if (config.isUseS3aSchema()) { + return String.format("s3a://%s/%s", config.getBucket(), config.getBaseKey()); + } return String.format("s3n://%s/%s", config.getBucket(), config.getBaseKey()); } @@ -78,7 +83,7 @@ public String getPathForHadoop(String dataSource) @Override public DataSegment push(final File indexFilesDir, final DataSegment inSegment) throws IOException { - final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), inSegment); + final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), getStorageDir(inSegment)); log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getIdentifier(), s3Path); @@ -107,16 +112,7 @@ public DataSegment call() throws Exception s3Client.putObject(outputBucket, toPush); final DataSegment outSegment = inSegment.withSize(indexSize) - .withLoadSpec( - ImmutableMap.of( - "type", - "s3_zip", - "bucket", - outputBucket, - "key", - toPush.getKey() - ) - ) + .withLoadSpec(makeLoadSpec(outputBucket, toPush.getKey())) .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); @@ -149,4 +145,30 @@ public DataSegment call() throws Exception throw Throwables.propagate(e); } } + + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + // remove the leading "/" + return makeLoadSpec(finalIndexZipFilePath.getHost(), finalIndexZipFilePath.getPath().substring(1)); + } + + /** + * Any change in loadSpec need to be reflected {@link io.druid.indexer.JobHelper#getURIFromSegment()} + * + */ + @SuppressWarnings("JavadocReference") + private Map makeLoadSpec(String bucket, String key) + { + return ImmutableMap.of( + "type", + "s3_zip", + "bucket", + bucket, + "key", + key, + "S3Schema", + config.isUseS3aSchema() ? "s3a" : "s3n" + ); + } } diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java index 7937ea339d17..28c5327da6cf 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java @@ -39,6 +39,9 @@ public class S3DataSegmentPusherConfig @JsonProperty @Min(0) private int maxListingLength = 1000; + // use s3n by default for backward compatibility + @JsonProperty + private boolean useS3aSchema = false; public void setBucket(String bucket) { @@ -60,6 +63,16 @@ public void setMaxListingLength(int maxListingLength) this.maxListingLength = maxListingLength; } + public boolean isUseS3aSchema() + { + return useS3aSchema; + } + + public void setUseS3aSchema(boolean useS3aSchema) + { + this.useS3aSchema = useS3aSchema; + } + public String getBucket() { return bucket; diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java index 0a3cdcebe0f4..83e931e6bd5f 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java @@ -108,6 +108,7 @@ public InputStream openStream() throws IOException } } + @Override public void pushTaskLog(final String taskid, final File logFile) throws IOException { final String taskKey = getTaskLogKey(taskid); diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java index 0fa86781d548..5d97e79ab25d 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java @@ -24,8 +24,6 @@ import com.google.common.base.Throwables; import io.druid.java.util.common.RetryUtils; -import io.druid.segment.loading.DataSegmentPusherUtil; -import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; import org.jets3t.service.StorageObjectsChunk; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -187,11 +185,11 @@ public void remove() }; } - public static String constructSegmentPath(String baseKey, DataSegment segment) + public static String constructSegmentPath(String baseKey, String storageDir) { return JOINER.join( baseKey.isEmpty() ? null : baseKey, - DataSegmentPusherUtil.getStorageDir(segment) + storageDir ) + "/index.zip"; } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 25950e334037..79a03559e8b6 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; +import org.easymock.EasyMock; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.junit.Assert; import org.junit.Test; @@ -54,6 +56,7 @@ public void testSerde() throws Exception ); Assert.assertEquals(factory, outputFact); + Assert.assertEquals(uris, outputFact.getUris()); } // This class is a workaround for the injectable value that StaticS3FirehoseFactory requires @@ -64,7 +67,7 @@ public TestStaticS3FirehoseFactory( @JsonProperty("uris") List uris ) { - super(null, uris); + super(EasyMock.niceMock(RestS3Service.class), uris, null, null, null, null, null, null); } } } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java index 8540e8a2ec19..4e74a4330434 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java @@ -158,7 +158,7 @@ public void testFailsToMoveMissing() throws Exception ), ImmutableMap.of("bucket", "DOES NOT EXIST", "baseKey", "baseKey2")); } - private class MockStorageService extends RestS3Service { + private static class MockStorageService extends RestS3Service { Map> storage = Maps.newHashMap(); boolean moved = false; diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherConfigTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherConfigTest.java index 44859c8a1ff3..8398909e8371 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherConfigTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherConfigTest.java @@ -39,7 +39,7 @@ public class S3DataSegmentPusherConfigTest public void testSerialization() throws IOException { String jsonConfig = "{\"bucket\":\"bucket1\",\"baseKey\":\"dataSource1\"," - +"\"disableAcl\":false,\"maxListingLength\":2000}"; + +"\"disableAcl\":false,\"maxListingLength\":2000,\"useS3aSchema\":false}"; S3DataSegmentPusherConfig config = jsonMapper.readValue(jsonConfig, S3DataSegmentPusherConfig.class); Assert.assertEquals(jsonConfig, jsonMapper.writeValueAsString(config)); @@ -50,7 +50,7 @@ public void testSerializationWithDefaults() throws IOException { String jsonConfig = "{\"bucket\":\"bucket1\",\"baseKey\":\"dataSource1\"}"; String expectedJsonConfig = "{\"bucket\":\"bucket1\",\"baseKey\":\"dataSource1\"," - +"\"disableAcl\":false,\"maxListingLength\":1000}"; + +"\"disableAcl\":false,\"maxListingLength\":1000,\"useS3aSchema\":false}"; S3DataSegmentPusherConfig config = jsonMapper.readValue(jsonConfig, S3DataSegmentPusherConfig.class); Assert.assertEquals(expectedJsonConfig, jsonMapper.writeValueAsString(config)); diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java index cf2d5c375555..dee6457a5a29 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentPusherTest.java @@ -44,7 +44,7 @@ */ public class S3DataSegmentPusherTest { - private class ValueContainer { + private static class ValueContainer { private T value; public T getValue() { diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java index b871ce985950..7c841f221ca9 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -37,6 +37,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -218,7 +219,7 @@ public String getInputType() @Override public List requiredFields() { - return Arrays.asList(fieldName); + return Collections.singletonList(fieldName); } @Override diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index fbb878efbc18..f231f9f005fd 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -75,6 +75,7 @@ public DetermineHashedPartitionsJob( this.config = config; } + @Override public boolean run() { try { @@ -82,7 +83,7 @@ public boolean run() * Group by (timestamp, dimensions) so we can correctly count dimension values as they would appear * in the final segment. */ - long startTime = System.currentTimeMillis(); + final long startTime = System.currentTimeMillis(); final Job groupByJob = Job.getInstance( new Configuration(), String.format("%s-determine_partitions_hashed-%s", config.getDataSource(), config.getIntervals()) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index e1f21df70e06..0792272db1fa 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -104,6 +104,7 @@ public DeterminePartitionsJob( this.config = config; } + @Override public boolean run() { try { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 77938c687021..29a01638a367 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -54,6 +54,7 @@ import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.loading.DataSegmentPusher; import io.druid.server.DruidNode; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.ShardSpec; @@ -92,9 +93,11 @@ public class HadoopDruidIndexerConfig public static final IndexMerger INDEX_MERGER; public static final IndexMergerV9 INDEX_MERGER_V9; public static final HadoopKerberosConfig HADOOP_KERBEROS_CONFIG; - + public static final DataSegmentPusher DATA_SEGMENT_PUSHER; private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing"; + + static { injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), @@ -118,6 +121,7 @@ public void configure(Binder binder) INDEX_MERGER = injector.getInstance(IndexMerger.class); INDEX_MERGER_V9 = injector.getInstance(IndexMergerV9.class); HADOOP_KERBEROS_CONFIG = injector.getInstance(HadoopKerberosConfig.class); + DATA_SEGMENT_PUSHER = injector.getInstance(DataSegmentPusher.class); } public static enum IndexJobCounters @@ -218,6 +222,7 @@ public static HadoopDruidIndexerConfig fromConfiguration(Configuration conf) private final Map shardSpecLookups = Maps.newHashMap(); private final Map> hadoopShardSpecLookup = Maps.newHashMap(); private final Granularity rollupGran; + private final List allowedHadoopPrefix; @JsonCreator public HadoopDruidIndexerConfig( @@ -254,6 +259,7 @@ public ShardSpec apply(HadoopyShardSpec input) } this.rollupGran = spec.getDataSchema().getGranularitySpec().getQueryGranularity(); + this.allowedHadoopPrefix = spec.getTuningConfig().getAllowedHadoopPrefix(); } @JsonProperty(value = "spec") @@ -592,4 +598,9 @@ public void verify() Preconditions.checkNotNull(schema.getIOConfig().getSegmentOutputPath(), "segmentOutputPath"); Preconditions.checkNotNull(schema.getTuningConfig().getVersion(), "version"); } + + public List getAllowedHadoopPrefix() + { + return allowedHadoopPrefix; + } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index 718dc6565c59..e64c0e7884df 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.druid.indexer.partitions.HashedPartitionsSpec; import io.druid.indexer.partitions.PartitionsSpec; @@ -66,7 +67,8 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DEFAULT_BUILD_V9_DIRECTLY, DEFAULT_NUM_BACKGROUND_PERSIST_THREADS, false, - false + false, + null ); } @@ -87,6 +89,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() private final int numBackgroundPersistThreads; private final boolean forceExtendableShardSpecs; private final boolean useExplicitVersion; + private final List allowedHadoopPrefix; @JsonCreator public HadoopTuningConfig( @@ -108,7 +111,8 @@ public HadoopTuningConfig( final @JsonProperty("buildV9Directly") Boolean buildV9Directly, final @JsonProperty("numBackgroundPersistThreads") Integer numBackgroundPersistThreads, final @JsonProperty("forceExtendableShardSpecs") boolean forceExtendableShardSpecs, - final @JsonProperty("useExplicitVersion") boolean useExplicitVersion + final @JsonProperty("useExplicitVersion") boolean useExplicitVersion, + final @JsonProperty("allowedHadoopPrefix") List allowedHadoopPrefix ) { this.workingPath = workingPath; @@ -135,6 +139,9 @@ public HadoopTuningConfig( this.forceExtendableShardSpecs = forceExtendableShardSpecs; Preconditions.checkArgument(this.numBackgroundPersistThreads >= 0, "Not support persistBackgroundCount < 0"); this.useExplicitVersion = useExplicitVersion; + this.allowedHadoopPrefix = allowedHadoopPrefix == null + ? ImmutableList.of("druid.storage.", "druid.javascript.") + : allowedHadoopPrefix; } @JsonProperty @@ -259,7 +266,8 @@ public HadoopTuningConfig withWorkingPath(String path) buildV9Directly, numBackgroundPersistThreads, forceExtendableShardSpecs, - useExplicitVersion + useExplicitVersion, + null ); } @@ -283,7 +291,8 @@ public HadoopTuningConfig withVersion(String ver) buildV9Directly, numBackgroundPersistThreads, forceExtendableShardSpecs, - useExplicitVersion + useExplicitVersion, + null ); } @@ -307,7 +316,14 @@ public HadoopTuningConfig withShardSpecs(Map> specs buildV9Directly, numBackgroundPersistThreads, forceExtendableShardSpecs, - useExplicitVersion + useExplicitVersion, + null ); } + + @JsonProperty + public List getAllowedHadoopPrefix() + { + return allowedHadoopPrefix; + } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 01fc8b5f73fb..5edb1f69b6e6 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -153,6 +153,7 @@ public IndexGeneratorStats getJobStats() return jobStats; } + @Override public boolean run() { try { @@ -165,6 +166,8 @@ public boolean run() JobHelper.injectSystemProperties(job); config.addJobProperties(job); + // inject druid properties like deep storage bindings + JobHelper.injectDruidProperties(job.getConfiguration(), config.getAllowedHadoopPrefix()); job.setMapperClass(IndexGeneratorMapper.class); job.setMapOutputValueClass(BytesWritable.class); @@ -740,20 +743,24 @@ indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator new Path(config.getSchema().getIOConfig().getSegmentOutputPath()), outputFS, segmentTemplate, - JobHelper.INDEX_ZIP + JobHelper.INDEX_ZIP, + config.DATA_SEGMENT_PUSHER ), JobHelper.makeFileNamePath( new Path(config.getSchema().getIOConfig().getSegmentOutputPath()), outputFS, segmentTemplate, - JobHelper.DESCRIPTOR_JSON + JobHelper.DESCRIPTOR_JSON, + config.DATA_SEGMENT_PUSHER ), JobHelper.makeTmpPath( new Path(config.getSchema().getIOConfig().getSegmentOutputPath()), outputFS, segmentTemplate, - context.getTaskAttemptID() - ) + context.getTaskAttemptID(), + config.DATA_SEGMENT_PUSHER + ), + config.DATA_SEGMENT_PUSHER ); Path descriptorPath = config.makeDescriptorInfoPath(segment); @@ -775,10 +782,7 @@ indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator FileUtils.deleteDirectory(file); } } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - catch (TimeoutException e) { + catch (ExecutionException | TimeoutException e) { throw Throwables.propagate(e); } finally { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java index 937c0db262db..677ecabc4bc3 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java @@ -86,36 +86,37 @@ public InputRow get() String k = aggFactory.getName(); writeString(k, out); - Aggregator agg = aggFactory.factorize( + try (Aggregator agg = aggFactory.factorize( IncrementalIndex.makeColumnSelectorFactory( VirtualColumns.EMPTY, aggFactory, supplier, true ) - ); - try { - agg.aggregate(); - } - catch (ParseException e) { - // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. - if (reportParseExceptions) { - throw new ParseException(e, "Encountered parse error for aggregator[%s]", k); + )) { + try { + agg.aggregate(); + } + catch (ParseException e) { + // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. + if (reportParseExceptions) { + throw new ParseException(e, "Encountered parse error for aggregator[%s]", k); + } + log.debug(e, "Encountered parse error, skipping aggregator[%s].", k); } - log.debug(e, "Encountered parse error, skipping aggregator[%s].", k); - } - String t = aggFactory.getTypeName(); + String t = aggFactory.getTypeName(); - if (t.equals("float")) { - out.writeFloat(agg.getFloat()); - } else if (t.equals("long")) { - WritableUtils.writeVLong(out, agg.getLong()); - } else { - //its a complex metric - Object val = agg.get(); - ComplexMetricSerde serde = getComplexMetricSerde(t); - writeBytes(serde.toBytes(val), out); + if (t.equals("float")) { + out.writeFloat(agg.getFloat()); + } else if (t.equals("long")) { + WritableUtils.writeVLong(out, agg.getLong()); + } else { + //its a complex metric + Object val = agg.get(); + ComplexMetricSerde serde = getComplexMetricSerde(t); + writeBytes(serde.toBytes(val), out); + } } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java index 20a8f7f6bf49..34b19245138f 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java @@ -23,7 +23,6 @@ import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.common.io.OutputSupplier; @@ -35,7 +34,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.segment.ProgressIndicator; import io.druid.segment.SegmentUtils; -import io.druid.segment.loading.DataSegmentPusherUtil; +import io.druid.segment.loading.DataSegmentPusher; import io.druid.timeline.DataSegment; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -45,6 +44,7 @@ import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryProxy; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.security.UserGroupInformation; @@ -310,6 +310,29 @@ public static void injectSystemProperties(Job job) injectSystemProperties(job.getConfiguration()); } + public static void injectDruidProperties(Configuration configuration, List listOfAllowedPrefix) + { + String mapJavaOpts = configuration.get(MRJobConfig.MAP_JAVA_OPTS); + String reduceJavaOpts = configuration.get(MRJobConfig.REDUCE_JAVA_OPTS); + + for (String propName : System.getProperties().stringPropertyNames()) { + for (String prefix : listOfAllowedPrefix) { + if (propName.startsWith(prefix)) { + mapJavaOpts = String.format("%s -D%s=%s", mapJavaOpts, propName, System.getProperty(propName)); + reduceJavaOpts = String.format("%s -D%s=%s", reduceJavaOpts, propName, System.getProperty(propName)); + break; + } + } + + } + if (!Strings.isNullOrEmpty(mapJavaOpts)) { + configuration.set(MRJobConfig.MAP_JAVA_OPTS, mapJavaOpts); + } + if (!Strings.isNullOrEmpty(reduceJavaOpts)) { + configuration.set(MRJobConfig.REDUCE_JAVA_OPTS, reduceJavaOpts); + } + } + public static Configuration injectSystemProperties(Configuration conf) { for (String propName : System.getProperties().stringPropertyNames()) { @@ -379,7 +402,8 @@ public static DataSegment serializeOutIndex( final File mergedBase, final Path finalIndexZipFilePath, final Path finalDescriptorPath, - final Path tmpPath + final Path tmpPath, + DataSegmentPusher dataSegmentPusher ) throws IOException { @@ -398,7 +422,6 @@ public long push() throws IOException progressable )) { size.set(zipAndCopyDir(mergedBase, outputStream, progressable)); - outputStream.flush(); } catch (IOException | RuntimeException exception) { log.error(exception, "Exception in retry loop"); @@ -413,43 +436,8 @@ public long push() throws IOException log.info("Zipped %,d bytes to [%s]", size.get(), tmpPath.toUri()); final URI indexOutURI = finalIndexZipFilePath.toUri(); - final ImmutableMap loadSpec; - // TODO: Make this a part of Pushers or Pullers - switch (outputFS.getScheme()) { - case "hdfs": - case "viewfs": - case "maprfs": - loadSpec = ImmutableMap.of( - "type", "hdfs", - "path", indexOutURI.toString() - ); - break; - case "gs": - loadSpec = ImmutableMap.of( - "type", "google", - "bucket", indexOutURI.getHost(), - "path", indexOutURI.getPath().substring(1) // remove the leading "/" - ); - break; - case "s3": - case "s3n": - loadSpec = ImmutableMap.of( - "type", "s3_zip", - "bucket", indexOutURI.getHost(), - "key", indexOutURI.getPath().substring(1) // remove the leading "/" - ); - break; - case "file": - loadSpec = ImmutableMap.of( - "type", "local", - "path", indexOutURI.getPath() - ); - break; - default: - throw new IAE("Unknown file system scheme [%s]", outputFS.getScheme()); - } final DataSegment finalSegment = segmentTemplate - .withLoadSpec(loadSpec) + .withLoadSpec(dataSegmentPusher.makeLoadSpec(indexOutURI)) .withSize(size.get()) .withBinaryVersion(SegmentUtils.getVersionFromDir(mergedBase)); @@ -576,77 +564,33 @@ private static void createNewZipEntry(ZipOutputStream out, File file) throws IOE out.putNextEntry(new ZipEntry(file.getName())); } - public static boolean isHdfs(FileSystem fs) - { - return "hdfs".equals(fs.getScheme()) || "viewfs".equals(fs.getScheme()) || "maprfs".equals(fs.getScheme()); - } - public static Path makeFileNamePath( final Path basePath, final FileSystem fs, final DataSegment segmentTemplate, - final String baseFileName + final String baseFileName, + DataSegmentPusher dataSegmentPusher ) { - final Path finalIndexZipPath; - final String segmentDir; - if (isHdfs(fs)) { - segmentDir = DataSegmentPusherUtil.getHdfsStorageDir(segmentTemplate); - finalIndexZipPath = new Path( - prependFSIfNullScheme(fs, basePath), - String.format( - "./%s/%d_%s", - segmentDir, - segmentTemplate.getShardSpec().getPartitionNum(), - baseFileName - ) - ); - } else { - segmentDir = DataSegmentPusherUtil.getStorageDir(segmentTemplate); - finalIndexZipPath = new Path( - prependFSIfNullScheme(fs, basePath), - String.format( - "./%s/%s", - segmentDir, - baseFileName - ) - ); - } - return finalIndexZipPath; + return new Path(prependFSIfNullScheme(fs, basePath), + dataSegmentPusher.makeIndexPathName(segmentTemplate, baseFileName)); } public static Path makeTmpPath( final Path basePath, final FileSystem fs, final DataSegment segmentTemplate, - final TaskAttemptID taskAttemptID + final TaskAttemptID taskAttemptID, + DataSegmentPusher dataSegmentPusher ) { - final String segmentDir; - - if (isHdfs(fs)) { - segmentDir = DataSegmentPusherUtil.getHdfsStorageDir(segmentTemplate); - return new Path( - prependFSIfNullScheme(fs, basePath), - String.format( - "./%s/%d_index.zip.%d", - segmentDir, - segmentTemplate.getShardSpec().getPartitionNum(), - taskAttemptID.getId() - ) - ); - } else { - segmentDir = DataSegmentPusherUtil.getStorageDir(segmentTemplate); - return new Path( - prependFSIfNullScheme(fs, basePath), - String.format( - "./%s/%d_index.zip.%d", - segmentDir, - segmentTemplate.getShardSpec().getPartitionNum(), - taskAttemptID.getId() - ) - ); - } + return new Path( + prependFSIfNullScheme(fs, basePath), + String.format("./%s.%d", + dataSegmentPusher.makeIndexPathName(segmentTemplate, JobHelper.INDEX_ZIP), + taskAttemptID.getId() + ) + ); } /** @@ -794,7 +738,12 @@ public static URI getURIFromSegment(DataSegment dataSegment) final String type = loadSpec.get("type").toString(); final URI segmentLocURI; if ("s3_zip".equals(type)) { - segmentLocURI = URI.create(String.format("s3n://%s/%s", loadSpec.get("bucket"), loadSpec.get("key"))); + if ("s3a".equals(loadSpec.get("S3Schema"))) { + segmentLocURI = URI.create(String.format("s3a://%s/%s", loadSpec.get("bucket"), loadSpec.get("key"))); + + } else { + segmentLocURI = URI.create(String.format("s3n://%s/%s", loadSpec.get("bucket"), loadSpec.get("key"))); + } } else if ("hdfs".equals(type)) { segmentLocURI = URI.create(loadSpec.get("path").toString()); } else if ("google".equals(type)) { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java index d9aa22238262..c703b0380f2a 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java @@ -23,16 +23,12 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import io.druid.collections.CountingMap; import io.druid.data.input.InputRow; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.JobHelper; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import org.apache.hadoop.conf.Configuration; @@ -56,8 +52,8 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.TreeSet; -import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class DatasourceInputFormat extends InputFormat { @@ -72,7 +68,10 @@ public List getSplits(JobContext context) throws IOException, Interr { Configuration conf = context.getConfiguration(); - String segmentsStr = Preconditions.checkNotNull(conf.get(CONF_INPUT_SEGMENTS), "No segments found to read"); + String segmentsStr = Preconditions.checkNotNull( + conf.get(CONF_INPUT_SEGMENTS), + "No segments found to read" + ); List segments = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( segmentsStr, new TypeReference>() @@ -91,7 +90,7 @@ public List getSplits(JobContext context) throws IOException, Interr for (WindowedDataSegment segment : segments) { totalSize += segment.getSegment().getSize(); } - int mapTask = ((JobConf)conf).getNumMapTasks(); + int mapTask = ((JobConf) conf).getNumMapTasks(); if (mapTask > 0) { maxSize = totalSize / mapTask; } @@ -159,7 +158,8 @@ public org.apache.hadoop.mapred.InputFormat get() //and not consider the splitting. //also without this, isSplitable(..) fails with NPE because compressionCodecs is not properly setup. @Override - protected boolean isSplitable(FileSystem fs, Path file) { + protected boolean isSplitable(FileSystem fs, Path file) + { return false; } @@ -191,64 +191,63 @@ private DatasourceInputSplit toDataSourceSplit( JobConf conf ) { - String[] locations = null; - try { - locations = getFrequentLocations(segments, fio, conf); - } - catch (Exception e) { - logger.error(e, "Exception thrown finding location of splits"); - } + String[] locations = getFrequentLocations(getLocations(segments, fio, conf)); + return new DatasourceInputSplit(segments, locations); } - private String[] getFrequentLocations( - List segments, - org.apache.hadoop.mapred.InputFormat fio, - JobConf conf - ) throws IOException + @VisibleForTesting + static Stream getLocations( + final List segments, + final org.apache.hadoop.mapred.InputFormat fio, + final JobConf conf + ) { - Iterable locations = Collections.emptyList(); - for (WindowedDataSegment segment : segments) { - FileInputFormat.setInputPaths(conf, new Path(JobHelper.getURIFromSegment(segment.getSegment()))); - for (org.apache.hadoop.mapred.InputSplit split : fio.getSplits(conf, 1)) { - locations = Iterables.concat(locations, Arrays.asList(split.getLocations())); - } - } - return getFrequentLocations(locations); + return segments.stream().sequential().flatMap( + (final WindowedDataSegment segment) -> { + FileInputFormat.setInputPaths( + conf, + new Path(JobHelper.getURIFromSegment(segment.getSegment())) + ); + try { + return Arrays.stream(fio.getSplits(conf, 1)).flatMap( + (final org.apache.hadoop.mapred.InputSplit split) -> { + try { + return Arrays.stream(split.getLocations()); + } + catch (final IOException e) { + logger.error(e, "Exception getting locations"); + return Stream.empty(); + } + } + ); + } + catch (final IOException e) { + logger.error(e, "Exception getting splits"); + return Stream.empty(); + } + } + ); } - private static String[] getFrequentLocations(Iterable hosts) + @VisibleForTesting + static String[] getFrequentLocations(final Stream locations) { - - final CountingMap counter = new CountingMap<>(); - for (String location : hosts) { - counter.add(location, 1); - } - - final TreeSet> sorted = Sets.>newTreeSet( - new Comparator>() - { - @Override - public int compare(Pair o1, Pair o2) - { - int compare = o2.lhs.compareTo(o1.lhs); // descending - if (compare == 0) { - compare = o1.rhs.compareTo(o2.rhs); // ascending - } - return compare; - } - } + final Map locationCountMap = locations.collect( + Collectors.groupingBy(location -> location, Collectors.counting()) ); - for (Map.Entry entry : counter.entrySet()) { - sorted.add(Pair.of(entry.getValue().get(), entry.getKey())); - } + final Comparator> valueComparator = + Map.Entry.comparingByValue(Comparator.reverseOrder()); - // use default replication factor, if possible - final List locations = Lists.newArrayListWithCapacity(3); - for (Pair frequent : Iterables.limit(sorted, 3)) { - locations.add(frequent.rhs); - } - return locations.toArray(new String[locations.size()]); + final Comparator> keyComparator = + Map.Entry.comparingByKey(); + + return locationCountMap + .entrySet().stream() + .sorted(valueComparator.thenComparing(keyComparator)) + .limit(3) + .map(Map.Entry::getKey) + .toArray(String[]::new); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/FSSpideringIterator.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/FSSpideringIterator.java index 22d558628d70..453fa7f5adae 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/FSSpideringIterator.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/FSSpideringIterator.java @@ -50,6 +50,7 @@ public static Iterable spiderIterable(final FileSystem fs, final Pat { return new Iterable() { + @Override public Iterator iterator() { return spiderPathPropagateExceptions(fs, path); @@ -72,6 +73,7 @@ public FSSpideringIterator( this.statii = statii; } + @Override public boolean hasNext() { if (statuses != null && !statuses.hasNext()) { @@ -81,6 +83,7 @@ public boolean hasNext() return index < statii.length; } + @Override public FileStatus next() { while (hasNext()) { @@ -98,6 +101,7 @@ public FileStatus next() throw new NoSuchElementException(); } + @Override public void remove() { throw new UnsupportedOperationException(); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java index 7908ba5b3c70..b227059ce6b9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/partitions/AbstractPartitionsSpec.java @@ -52,18 +52,21 @@ public AbstractPartitionsSpec( ); } + @Override @JsonProperty public long getTargetPartitionSize() { return targetPartitionSize; } + @Override @JsonProperty public long getMaxPartitionSize() { return maxPartitionSize; } + @Override @JsonProperty public boolean isAssumeGrouped() { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/path/GranularUnprocessedPathSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/path/GranularUnprocessedPathSpec.java index 10f444a723a2..8bacae47377a 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/path/GranularUnprocessedPathSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/path/GranularUnprocessedPathSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import io.druid.java.util.common.granularity.Granularity; import io.druid.indexer.HadoopDruidIndexerConfig; @@ -74,13 +75,11 @@ public Job addInputPaths(HadoopDruidIndexerConfig config, Job job) throws IOExce final FileSystem fs = betaInput.getFileSystem(job.getConfiguration()); final Granularity segmentGranularity = config.getGranularitySpec().getSegmentGranularity(); - Map inputModifiedTimes = new TreeMap<>( - Comparators.inverse(Comparators.comparable()) - ); + Map inputModifiedTimes = new TreeMap<>(Ordering.natural().reverse()); for (FileStatus status : FSSpideringIterator.spiderIterable(fs, betaInput)) { final DateTime key = segmentGranularity.toDate(status.getPath().toString()); - final Long currVal = inputModifiedTimes.get(key); + final Long currVal = inputModifiedTimes.get(key.getMillis()); final long mTime = status.getModificationTime(); inputModifiedTimes.put(key.getMillis(), currVal == null ? mTime : Math.max(currVal, mTime)); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java index 3675b2a84e57..ab92c7032051 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java @@ -556,20 +556,24 @@ protected void map( baseOutputPath, outputFS, finalSegmentTemplate, - JobHelper.INDEX_ZIP + JobHelper.INDEX_ZIP, + config.DATA_SEGMENT_PUSHER ), JobHelper.makeFileNamePath( baseOutputPath, outputFS, finalSegmentTemplate, - JobHelper.DESCRIPTOR_JSON + JobHelper.DESCRIPTOR_JSON, + config.DATA_SEGMENT_PUSHER ), JobHelper.makeTmpPath( baseOutputPath, outputFS, finalSegmentTemplate, - context.getTaskAttemptID() - ) + context.getTaskAttemptID(), + config.DATA_SEGMENT_PUSHER + ), + config.DATA_SEGMENT_PUSHER ); context.progress(); context.setStatus("Finished PUSH"); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopDruidConverterConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopDruidConverterConfig.java index 57f2b8f8c51c..a3574f16cb25 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopDruidConverterConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopDruidConverterConfig.java @@ -37,6 +37,7 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; +import io.druid.segment.loading.DataSegmentPusher; import io.druid.server.DruidNode; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -53,6 +54,7 @@ public class HadoopDruidConverterConfig public static final ObjectMapper jsonMapper; public static final IndexIO INDEX_IO; public static final IndexMerger INDEX_MERGER; + public static final DataSegmentPusher DATA_SEGMENT_PUSHER; private static final Injector injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), @@ -75,6 +77,7 @@ public void configure(Binder binder) jsonMapper.registerSubtypes(HadoopDruidConverterConfig.class); INDEX_IO = injector.getInstance(IndexIO.class); INDEX_MERGER = injector.getInstance(IndexMerger.class); + DATA_SEGMENT_PUSHER = injector.getInstance(DataSegmentPusher.class); } private static final TypeReference> mapTypeReference = new TypeReference>() diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index 7201bd119133..954910470246 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -346,7 +346,9 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig(Map data(){ + public static Collection data() + { int[] first = new int[1]; Arrays.fill(first, 13); int[] second = new int[6]; @@ -67,7 +68,7 @@ public static Collection data(){ return Arrays.asList( new Object[][]{ { - DetermineHashedPartitionsJobTest.class.getClass().getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(), + DetermineHashedPartitionsJobTest.class.getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(), 1L, "2011-04-10T00:00:00.000Z/2011-04-11T00:00:00.000Z", 0, @@ -75,7 +76,7 @@ public static Collection data(){ first }, { - DetermineHashedPartitionsJobTest.class.getClass().getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(), + DetermineHashedPartitionsJobTest.class.getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(), 100L, "2011-04-10T00:00:00.000Z/2011-04-16T00:00:00.000Z", 0, @@ -83,7 +84,7 @@ public static Collection data(){ second }, { - DetermineHashedPartitionsJobTest.class.getClass().getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(), + DetermineHashedPartitionsJobTest.class.getResource("/druid.test.data.with.duplicate.rows.tsv").getPath(), 1L, "2011-04-10T00:00:00.000Z/2011-04-16T00:00:00.000Z", 0, @@ -116,7 +117,12 @@ public DetermineHashedPartitionsJobTest( new DelimitedParseSpec( new TimestampSpec("ts", null, null), new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("market", "quality", "placement", "placementish")), + DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "market", + "quality", + "placement", + "placementish" + )), null, null ), @@ -129,7 +135,9 @@ public DetermineHashedPartitionsJobTest( "placement", "placementish", "index" - ) + ), + false, + 0 ), null ), @@ -169,14 +177,16 @@ public DetermineHashedPartitionsJobTest( null, null, false, - false + false, + null ) ); this.indexerConfig = new HadoopDruidIndexerConfig(ingestionSpec); } @Test - public void testDetermineHashedPartitions(){ + public void testDetermineHashedPartitions() + { DetermineHashedPartitionsJob determineHashedPartitionsJob = new DetermineHashedPartitionsJob(indexerConfig); determineHashedPartitionsJob.run(); Map> shardSpecs = indexerConfig.getSchema().getTuningConfig().getShardSpecs(); @@ -184,8 +194,8 @@ public void testDetermineHashedPartitions(){ expectedNumTimeBuckets, shardSpecs.entrySet().size() ); - int i=0; - for(Map.Entry> entry : shardSpecs.entrySet()) { + int i = 0; + for (Map.Entry> entry : shardSpecs.entrySet()) { Assert.assertEquals( expectedNumOfShards[i++], entry.getValue().size(), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java index 9fc2e8eb0400..96d3cf9092c7 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java @@ -227,7 +227,9 @@ public DeterminePartitionsJobTest( new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")), null, null), null, - ImmutableList.of("timestamp", "host", "country", "visited_num") + ImmutableList.of("timestamp", "host", "country", "visited_num"), + false, + 0 ), null ), @@ -267,7 +269,8 @@ public DeterminePartitionsJobTest( null, null, false, - false + false, + null ) ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index 4b6cc7ce8a36..3c1ffdc1fdd8 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -31,15 +30,8 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; -import io.druid.timeline.DataSegment; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; -import io.druid.timeline.partition.NumberedShardSpec; -import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.TaskType; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -58,229 +50,6 @@ public class HadoopDruidIndexerConfigTest jsonMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper)); } - public static T jsonReadWriteRead(String s, Class klass) - { - try { - return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - @Test - public void shouldMakeHDFSCompliantSegmentOutputPath() - { - HadoopIngestionSpec schema; - - try { - schema = jsonReadWriteRead( - "{\n" - + " \"dataSchema\": {\n" - + " \"dataSource\": \"source\",\n" - + " \"metricsSpec\": [],\n" - + " \"granularitySpec\": {\n" - + " \"type\": \"uniform\",\n" - + " \"segmentGranularity\": \"hour\",\n" - + " \"intervals\": [\"2012-07-10/P1D\"]\n" - + " }\n" - + " },\n" - + " \"ioConfig\": {\n" - + " \"type\": \"hadoop\",\n" - + " \"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\"\n" - + " }\n" - + "}", - HadoopIngestionSpec.class - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - - HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( - schema.withTuningConfig( - schema.getTuningConfig() - .withVersion( - "some:brand:new:version" - ) - ) - ); - - Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); - Path path = JobHelper.makeFileNamePath( - new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), - new DistributedFileSystem(), - new DataSegment( - cfg.getSchema().getDataSchema().getDataSource(), - cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), - cfg.getSchema().getTuningConfig().getVersion(), - null, - null, - null, - new NumberedShardSpec(bucket.partitionNum, 5000), - -1, - -1 - ), - JobHelper.INDEX_ZIP - ); - Assert.assertEquals( - "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version" - + "/4712_index.zip", - path.toString() - ); - - path = JobHelper.makeFileNamePath( - new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), - new DistributedFileSystem(), - new DataSegment( - cfg.getSchema().getDataSchema().getDataSource(), - cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), - cfg.getSchema().getTuningConfig().getVersion(), - null, - null, - null, - new NumberedShardSpec(bucket.partitionNum, 5000), - -1, - -1 - ), - JobHelper.DESCRIPTOR_JSON - ); - Assert.assertEquals( - "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version" - + "/4712_descriptor.json", - path.toString() - ); - - path = JobHelper.makeTmpPath( - new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), - new DistributedFileSystem(), - new DataSegment( - cfg.getSchema().getDataSchema().getDataSource(), - cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), - cfg.getSchema().getTuningConfig().getVersion(), - null, - null, - null, - new NumberedShardSpec(bucket.partitionNum, 5000), - -1, - -1 - ), - new TaskAttemptID("abc", 123, TaskType.REDUCE, 1, 0) - ); - Assert.assertEquals( - "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version" - + "/4712_index.zip.0", - path.toString() - ); - - } - - @Test - public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() - { - final HadoopIngestionSpec schema; - - try { - schema = jsonReadWriteRead( - "{\n" - + " \"dataSchema\": {\n" - + " \"dataSource\": \"the:data:source\",\n" - + " \"metricsSpec\": [],\n" - + " \"granularitySpec\": {\n" - + " \"type\": \"uniform\",\n" - + " \"segmentGranularity\": \"hour\",\n" - + " \"intervals\": [\"2012-07-10/P1D\"]\n" - + " }\n" - + " },\n" - + " \"ioConfig\": {\n" - + " \"type\": \"hadoop\",\n" - + " \"segmentOutputPath\": \"/tmp/dru:id/data:test\"\n" - + " }\n" - + "}", - HadoopIngestionSpec.class - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - - HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( - schema.withTuningConfig( - schema.getTuningConfig() - .withVersion( - "some:brand:new:version" - ) - ) - ); - - Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); - Path path = JobHelper.makeFileNamePath( - new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), - new LocalFileSystem(), - new DataSegment( - cfg.getSchema().getDataSchema().getDataSource(), - cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), - cfg.getSchema().getTuningConfig().getVersion(), - null, - null, - null, - new NumberedShardSpec(bucket.partitionNum, 5000), - -1, - -1 - ), - JobHelper.INDEX_ZIP - ); - Assert.assertEquals( - "file:/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:" - + "version/4712/index.zip", - path.toString() - ); - - path = JobHelper.makeFileNamePath( - new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), - new LocalFileSystem(), - new DataSegment( - cfg.getSchema().getDataSchema().getDataSource(), - cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), - cfg.getSchema().getTuningConfig().getVersion(), - null, - null, - null, - new NumberedShardSpec(bucket.partitionNum, 5000), - -1, - -1 - ), - JobHelper.DESCRIPTOR_JSON - ); - Assert.assertEquals( - "file:/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:" - + "version/4712/descriptor.json", - path.toString() - ); - - path = JobHelper.makeTmpPath( - new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), - new LocalFileSystem(), - new DataSegment( - cfg.getSchema().getDataSchema().getDataSource(), - cfg.getSchema().getDataSchema().getGranularitySpec().bucketInterval(bucket.time).get(), - cfg.getSchema().getTuningConfig().getVersion(), - null, - null, - null, - new NumberedShardSpec(bucket.partitionNum, 5000), - -1, - -1 - ), - new TaskAttemptID("abc", 123, TaskType.REDUCE, 1, 0) - ); - Assert.assertEquals( - "file:/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:" - + "version/4712/4712_index.zip.0", - path.toString() - ); - - } @Test public void testHashedBucketSelection() @@ -325,7 +94,8 @@ public void testHashedBucketSelection() null, null, false, - false + false, + null ) ); HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec(spec); @@ -397,7 +167,8 @@ public void testNoneShardSpecBucketSelection() null, null, false, - false + false, + null ) ); HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec(spec); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java index c36af52fc950..753379ba709b 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java @@ -57,7 +57,8 @@ public void testSerde() throws Exception null, null, true, - true + true, + null ); HadoopTuningConfig actual = jsonReadWriteRead(jsonMapper.writeValueAsString(expected), HadoopTuningConfig.class); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java index 087c13d790c7..b657e2e45be8 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java @@ -46,6 +46,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -245,14 +246,14 @@ public void testMultipleRowsNotMerged() throws Exception InputRow capturedRow1 = InputRowSerde.fromBytes(captureVal1.getValue().getBytes(), aggregators); Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow1.getDimensions()); - Assert.assertEquals(Arrays.asList("host1"), capturedRow1.getDimension("host")); + Assert.assertEquals(Collections.singletonList("host1"), capturedRow1.getDimension("host")); Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow1.getDimension("keywords")); Assert.assertEquals(10, capturedRow1.getLongMetric("visited_sum")); Assert.assertEquals(1.0, (Double)HyperUniquesAggregatorFactory.estimateCardinality(capturedRow1.getRaw("unique_hosts")), 0.001); InputRow capturedRow2 = InputRowSerde.fromBytes(captureVal2.getValue().getBytes(), aggregators); Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow2.getDimensions()); - Assert.assertEquals(Arrays.asList("host2"), capturedRow2.getDimension("host")); + Assert.assertEquals(Collections.singletonList("host2"), capturedRow2.getDimension("host")); Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow2.getDimension("keywords")); Assert.assertEquals(5, capturedRow2.getLongMetric("visited_sum")); Assert.assertEquals(1.0, (Double)HyperUniquesAggregatorFactory.estimateCardinality(capturedRow2.getRaw("unique_hosts")), 0.001); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index 250b9dd8b045..190744358573 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -142,7 +142,9 @@ public static Collection constructFeed() new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, - ImmutableList.of("timestamp", "host", "visited_num") + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 ), null ), @@ -188,7 +190,9 @@ public static Collection constructFeed() new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, - ImmutableList.of("timestamp", "host", "visited_num") + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 ) ), null, @@ -233,7 +237,9 @@ public static Collection constructFeed() new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, - ImmutableList.of("timestamp", "host", "visited_num") + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 ), null ), @@ -289,7 +295,9 @@ public static Collection constructFeed() new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, - ImmutableList.of("timestamp", "host", "visited_num") + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 ) ), null, @@ -517,7 +525,8 @@ public void setUp() throws Exception buildV9Directly, null, forceExtendableShardSpecs, - false + false, + null ) ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java index d1455fe0abb9..fcd7ff75103f 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/InputRowSerdeTest.java @@ -26,10 +26,14 @@ import io.druid.hll.HyperLogLogCollector; import io.druid.jackson.AggregatorsModule; import io.druid.java.util.common.parsers.ParseException; +import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregator; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.segment.ColumnSelectorFactory; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -65,7 +69,14 @@ public InputRowSerdeTest() @Test public void testSerde() { - + // Prepare the mocks & set close() call count expectation to 1 + final Aggregator mockedAggregator = EasyMock.createMock(DoubleSumAggregator.class); + EasyMock.expect(mockedAggregator.getFloat()).andReturn(0f).times(1); + mockedAggregator.aggregate(); + EasyMock.expectLastCall().times(1); + mockedAggregator.close(); + EasyMock.expectLastCall().times(1); + EasyMock.replay(mockedAggregator); InputRow in = new MapBasedInputRow( timestamp, @@ -78,7 +89,14 @@ public void testSerde() new DoubleSumAggregatorFactory("m1out", "m1"), new LongSumAggregatorFactory("m2out", "m2"), new HyperUniquesAggregatorFactory("m3out", "m3"), - new LongSumAggregatorFactory("unparseable", "m3") // Unparseable from String to Long + new LongSumAggregatorFactory("unparseable", "m3"), // Unparseable from String to Long + new DoubleSumAggregatorFactory("mockedAggregator", "m4") { + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return mockedAggregator; + } + } }; byte[] data = InputRowSerde.toBytes(in, aggregatorFactories, false); // Ignore Unparseable aggregator @@ -96,6 +114,7 @@ public void testSerde() Assert.assertEquals(1, ((HyperLogLogCollector) out.getRaw("m3out")).estimateCardinality(), 0.001); Assert.assertEquals(0L, out.getLongMetric("unparseable")); + EasyMock.verify(mockedAggregator); } @Test(expected = ParseException.class) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index 8af6c470ff69..83f4449f20c6 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -77,7 +77,9 @@ public void setup() throws Exception new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, - ImmutableList.of("timestamp", "host", "visited_num") + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 ), null ), @@ -123,7 +125,8 @@ public void setup() throws Exception null, null, false, - false + false, + null ) ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/UtilsCompressionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/UtilsCompressionTest.java index 750298d6c3d8..87ff6e6a0c4e 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/UtilsCompressionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/UtilsCompressionTest.java @@ -86,13 +86,14 @@ public void tearDown() tmpFolder.delete(); } - @Test public void testExistsCompressedFile() throws IOException + @Test + public void testExistsCompressedFile() throws IOException { - boolean expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPathWithoutExtension); + boolean expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPathWithoutExtension); Assert.assertTrue("Should be true since file is created", expected); tmpFolder.delete(); - expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPathWithoutExtension); - Assert.assertFalse("Should be false since file is deleted",expected); + expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPathWithoutExtension); + Assert.assertFalse("Should be false since file is deleted", expected); } @Test @@ -100,11 +101,11 @@ public void testCompressedOpenInputStream() throws IOException { boolean overwrite = true; OutputStream outStream = codec.createOutputStream(defaultFileSystem.create(tmpPathWithExtension, overwrite)); - writeStingToOutputStream(DUMMY_STRING,outStream); + writeStingToOutputStream(DUMMY_STRING, outStream); InputStream inStream = Utils.openInputStream(mockJobContext, tmpPathWithoutExtension); - Assert.assertNotNull("Input stream should not be Null",inStream); + Assert.assertNotNull("Input stream should not be Null", inStream); String actual = new String(ByteStreams.toByteArray(inStream), StandardCharsets.UTF_8.toString()); - Assert.assertEquals("Strings not matching", DUMMY_STRING,actual); + Assert.assertEquals("Strings not matching", DUMMY_STRING, actual); inStream.close(); } @@ -112,12 +113,12 @@ public void testCompressedOpenInputStream() throws IOException public void testCompressedMakePathAndOutputStream() throws IOException { boolean overwrite = true; - OutputStream outStream = Utils.makePathAndOutputStream(mockJobContext,tmpPathWithoutExtension, overwrite); - Assert.assertNotNull("Output stream should not be null",outStream); - writeStingToOutputStream(DUMMY_STRING,outStream); + OutputStream outStream = Utils.makePathAndOutputStream(mockJobContext, tmpPathWithoutExtension, overwrite); + Assert.assertNotNull("Output stream should not be null", outStream); + writeStingToOutputStream(DUMMY_STRING, outStream); InputStream inStream = codec.createInputStream(defaultFileSystem.open(tmpPathWithExtension)); String actual = new String(ByteStreams.toByteArray(inStream), StandardCharsets.UTF_8.toString()); - Assert.assertEquals("Strings not matching", DUMMY_STRING,actual); + Assert.assertEquals("Strings not matching", DUMMY_STRING, actual); inStream.close(); } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/UtilsTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/UtilsTest.java index ad638da6e034..70d024ae7ad9 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/UtilsTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/UtilsTest.java @@ -22,9 +22,7 @@ import com.google.common.base.Function; import com.google.common.collect.Maps; import com.google.common.io.ByteStreams; - import io.druid.java.util.common.ISE; - import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -65,9 +63,10 @@ public class UtilsTest @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - private class CreateValueFromKey implements Function + private static class CreateValueFromKey implements Function { - @Override public Object apply(Object input) + @Override + public Object apply(Object input) { return input.toString() + DUMMY_STRING; } @@ -82,7 +81,7 @@ public void setUp() throws IOException EasyMock.replay(mockJobContext); setOfKeys = new HashSet(); - setOfKeys.addAll(new ArrayList<>(Arrays.asList("key1","key2","key3"))); + setOfKeys.addAll(new ArrayList<>(Arrays.asList("key1", "key2", "key3"))); expectedMap = (Map) Maps.asMap(setOfKeys, new CreateValueFromKey()); tmpFile = tmpFolder.newFile(TMP_FILE_NAME); @@ -99,20 +98,20 @@ public void tearDown() @Test public void testExistsPlainFile() throws IOException { - boolean expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPath); - Assert.assertTrue("Should be true since file is created",expected); + boolean expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPath); + Assert.assertTrue("Should be true since file is created", expected); tmpFolder.delete(); - expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPath); - Assert.assertFalse("Should be false since file is deleted",expected); + expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPath); + Assert.assertFalse("Should be false since file is deleted", expected); EasyMock.verify(mockJobContext); } @Test public void testPlainStoreThenGetStats() throws IOException { - Utils.storeStats(mockJobContext, tmpPath,expectedMap); + Utils.storeStats(mockJobContext, tmpPath, expectedMap); Map actualMap = Utils.getStats(mockJobContext, tmpPath); - Assert.assertThat(actualMap,Is.is(actualMap)); + Assert.assertThat(actualMap, Is.is(actualMap)); EasyMock.verify(mockJobContext); } @@ -120,7 +119,7 @@ public void testPlainStoreThenGetStats() throws IOException public void testExceptionInMakePathAndOutputStream() throws IOException { boolean overwrite = false; - Utils.makePathAndOutputStream(mockJobContext,tmpPath,overwrite); + Utils.makePathAndOutputStream(mockJobContext, tmpPath, overwrite); } @Test diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java index 3ad982369df3..616d364a0452 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceInputFormatTest.java @@ -53,6 +53,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Stream; /** */ @@ -323,4 +324,107 @@ public void testGetRecordReader() throws Exception { Assert.assertTrue(new DatasourceInputFormat().createRecordReader(null, null) instanceof DatasourceRecordReader); } + + @Test + public void testGetFrequentLocationsEmpty() + { + Assert.assertArrayEquals( + new String[0], + DatasourceInputFormat.getFrequentLocations(Stream.empty()) + ); + } + + @Test + public void testGetFrequentLocationsLessThan3() + { + Assert.assertArrayEquals( + new String[]{"s1", "s2"}, + DatasourceInputFormat.getFrequentLocations(Stream.of("s2", "s1")) + ); + } + + @Test + public void testGetFrequentLocationsMoreThan3() + { + Assert.assertArrayEquals( + new String[]{"s3", "s1", "s2"}, + DatasourceInputFormat.getFrequentLocations( + Stream.of("s3", "e", "s2", "s3", "s4", "s3", "s1", "s3", "s2", "s1") + ) + ); + } + + @Test + public void testGetLocationsInputFormatException() throws IOException + { + final org.apache.hadoop.mapred.InputFormat fio = EasyMock.mock( + org.apache.hadoop.mapred.InputFormat.class + ); + + EasyMock.expect(fio.getSplits(config, 1)).andThrow(new IOException("testing")); + EasyMock.replay(fio); + + Assert.assertEquals( + 0, + DatasourceInputFormat.getLocations(segments.subList(0, 1), fio, config).count() + ); + } + + @Test + public void testGetLocationsSplitException() throws IOException + { + final org.apache.hadoop.mapred.InputFormat fio = EasyMock.mock( + org.apache.hadoop.mapred.InputFormat.class + ); + + final org.apache.hadoop.mapred.InputSplit split = EasyMock.mock( + org.apache.hadoop.mapred.InputSplit.class + ); + + EasyMock.expect(fio.getSplits(config, 1)).andReturn( + new org.apache.hadoop.mapred.InputSplit[] {split} + ); + EasyMock.expect(split.getLocations()).andThrow(new IOException("testing")); + + EasyMock.replay(fio, split); + + Assert.assertEquals( + 0, + DatasourceInputFormat.getLocations(segments.subList(0, 1), fio, config).count() + ); + } + + @Test + public void testGetLocations() throws IOException + { + final org.apache.hadoop.mapred.InputFormat fio = EasyMock.mock( + org.apache.hadoop.mapred.InputFormat.class + ); + + final org.apache.hadoop.mapred.InputSplit split = EasyMock.mock( + org.apache.hadoop.mapred.InputSplit.class + ); + + EasyMock.expect(fio.getSplits(config, 1)).andReturn( + new org.apache.hadoop.mapred.InputSplit[] {split} + ); + EasyMock.expect(split.getLocations()).andReturn(new String[] {"s1", "s2"}); + + EasyMock.expect(fio.getSplits(config, 1)).andReturn( + new org.apache.hadoop.mapred.InputSplit[] {split} + ); + EasyMock.expect(split.getLocations()).andReturn(new String[] {"s3"}); + + EasyMock.expect(fio.getSplits(config, 1)).andReturn( + new org.apache.hadoop.mapred.InputSplit[] {split} + ); + EasyMock.expect(split.getLocations()).andReturn(new String[] {"s4", "s2"}); + + EasyMock.replay(fio, split); + + Assert.assertArrayEquals( + new String[] {"s1", "s2", "s3", "s4", "s2"}, + DatasourceInputFormat.getLocations(segments, fio, config).toArray(String[]::new) + ); + } } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java index f1c7d558c909..a8cb0216705e 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java @@ -265,7 +265,9 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig() new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(null, null, null), null, - ImmutableList.of("timestamp", "host", "visited") + ImmutableList.of("timestamp", "host", "visited"), + false, + 0 ), null ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java index ba86ce2708d4..154e03399e58 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java @@ -71,7 +71,8 @@ public class GranularityPathSpecTest null, null, false, - false + false, + null ); private GranularityPathSpec granularityPathSpec; diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 4b9adba54c4f..4a3eceedc56f 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -164,7 +164,9 @@ public InputStream openStream() throws IOException new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList(TestIndex.DIMENSIONS)), null, null), "\t", "\u0001", - Arrays.asList(TestIndex.COLUMNS) + Arrays.asList(TestIndex.COLUMNS), + false, + 0 ), null ), @@ -207,7 +209,8 @@ public InputStream openStream() throws IOException null, null, false, - false + false, + null ) ) ); diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java index b25d88674185..2ead3b2c4e45 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java @@ -25,7 +25,6 @@ import io.druid.segment.indexing.DataSchema; import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import org.joda.time.DateTime; import java.io.IOException; @@ -45,7 +44,6 @@ public ActionBasedSegmentAllocator( @Override public SegmentIdentifier allocate( - final DateTime timestamp, final InputRow row, final String sequenceName, final String previousSegmentId @@ -54,7 +52,7 @@ public SegmentIdentifier allocate( return taskActionClient.submit( new SegmentAllocateAction( dataSchema.getDataSource(), - timestamp, + row.getTimestamp(), dataSchema.getGranularitySpec().getQueryGranularity(), dataSchema.getGranularitySpec().getSegmentGranularity(), sequenceName, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/SegmentLoaderFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/SegmentLoaderFactory.java index 07f6f4f5d262..5cd7ee5d2e9a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/SegmentLoaderFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/SegmentLoaderFactory.java @@ -26,7 +26,7 @@ import io.druid.segment.loading.StorageLocationConfig; import java.io.File; -import java.util.Arrays; +import java.util.Collections; /** */ @@ -45,7 +45,7 @@ public SegmentLoaderFactory( public SegmentLoader manufacturate(File storageDir) { return loader.withConfig( - new SegmentLoaderConfig().withLocations(Arrays.asList(new StorageLocationConfig().setPath(storageDir))) + new SegmentLoaderConfig().withLocations(Collections.singletonList(new StorageLocationConfig().setPath(storageDir))) ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index d9a690eb65ba..8671432b77be 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -46,6 +46,7 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -70,6 +71,7 @@ public class TaskToolbox private final DataSegmentArchiver dataSegmentArchiver; private final DataSegmentMover dataSegmentMover; private final DataSegmentAnnouncer segmentAnnouncer; + private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentHandoffNotifierFactory handoffNotifierFactory; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; private final MonitorScheduler monitorScheduler; @@ -93,6 +95,7 @@ public TaskToolbox( DataSegmentMover dataSegmentMover, DataSegmentArchiver dataSegmentArchiver, DataSegmentAnnouncer segmentAnnouncer, + DataSegmentServerAnnouncer serverAnnouncer, SegmentHandoffNotifierFactory handoffNotifierFactory, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, ExecutorService queryExecutorService, @@ -116,6 +119,7 @@ public TaskToolbox( this.dataSegmentMover = dataSegmentMover; this.dataSegmentArchiver = dataSegmentArchiver; this.segmentAnnouncer = segmentAnnouncer; + this.serverAnnouncer = serverAnnouncer; this.handoffNotifierFactory = handoffNotifierFactory; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; this.queryExecutorService = queryExecutorService; @@ -170,6 +174,11 @@ public DataSegmentAnnouncer getSegmentAnnouncer() return segmentAnnouncer; } + public DataSegmentServerAnnouncer getDataSegmentServerAnnouncer() + { + return serverAnnouncer; + } + public SegmentHandoffNotifierFactory getSegmentHandoffNotifierFactory() { return handoffNotifierFactory; @@ -253,4 +262,19 @@ public CacheConfig getCacheConfig() public IndexMergerV9 getIndexMergerV9() { return indexMergerV9; } + + public File getFirehoseTemporaryDir() + { + return new File(taskWorkDir, "firehose"); + } + + public File getMergeDir() + { + return new File(taskWorkDir, "merge"); + } + + public File getPersistDir() + { + return new File(taskWorkDir, "persist"); + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 291fbde00b84..7527d1b8761b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -40,6 +40,7 @@ import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import java.io.File; import java.util.concurrent.ExecutorService; @@ -57,6 +58,7 @@ public class TaskToolboxFactory private final DataSegmentMover dataSegmentMover; private final DataSegmentArchiver dataSegmentArchiver; private final DataSegmentAnnouncer segmentAnnouncer; + private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentHandoffNotifierFactory handoffNotifierFactory; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; private final ExecutorService queryExecutorService; @@ -79,6 +81,7 @@ public TaskToolboxFactory( DataSegmentMover dataSegmentMover, DataSegmentArchiver dataSegmentArchiver, DataSegmentAnnouncer segmentAnnouncer, + DataSegmentServerAnnouncer serverAnnouncer, SegmentHandoffNotifierFactory handoffNotifierFactory, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, @Processing ExecutorService queryExecutorService, @@ -100,6 +103,7 @@ public TaskToolboxFactory( this.dataSegmentMover = dataSegmentMover; this.dataSegmentArchiver = dataSegmentArchiver; this.segmentAnnouncer = segmentAnnouncer; + this.serverAnnouncer = serverAnnouncer; this.handoffNotifierFactory = handoffNotifierFactory; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; this.queryExecutorService = queryExecutorService; @@ -126,6 +130,7 @@ public TaskToolbox build(Task task) dataSegmentMover, dataSegmentArchiver, segmentAnnouncer, + serverAnnouncer, handoffNotifierFactory, queryRunnerFactoryConglomerate, queryExecutorService, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java index 6d05a9cf6f73..2d61f0639242 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java @@ -47,6 +47,7 @@ public Interval getInterval() return interval; } + @Override public TypeReference getReturnTypeReference() { return new TypeReference() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockListAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockListAction.java index 40ea12abe388..06cea4688102 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockListAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockListAction.java @@ -27,6 +27,7 @@ public class LockListAction implements TaskAction> { + @Override public TypeReference> getReturnTypeReference() { return new TypeReference>() {}; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java index 8a3ffa6c0a10..f68e780ed6f5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java @@ -45,6 +45,7 @@ public Interval getInterval() return interval; } + @Override public TypeReference getReturnTypeReference() { return new TypeReference() {}; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java index 06243a8fc874..28cfd6ef22ae 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java @@ -46,6 +46,7 @@ public Interval getInterval() return interval; } + @Override public TypeReference getReturnTypeReference() { return new TypeReference() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java index 2e3dbd5d9418..d73c073c9dda 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java @@ -56,6 +56,7 @@ public Set getSegments() return segments; } + @Override public TypeReference> getReturnTypeReference() { return new TypeReference>() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUnusedAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUnusedAction.java index 83585a9869bf..1836d2cf9aee 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUnusedAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUnusedAction.java @@ -60,6 +60,7 @@ public Interval getInterval() return interval; } + @Override public TypeReference> getReturnTypeReference() { return new TypeReference>() {}; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUsedAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUsedAction.java index 4cf77449c3b8..6e4e52c6b5d4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUsedAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentListUsedAction.java @@ -76,6 +76,7 @@ public List getIntervals() return intervals; } + @Override public TypeReference> getReturnTypeReference() { return new TypeReference>() {}; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index de5670f0d7fc..6162dce4e8ce 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -51,6 +51,7 @@ public Set getSegments() return segments; } + @Override public TypeReference getReturnTypeReference() { return new TypeReference() {}; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java index 3f195c563f5a..2d078ebfaeed 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentNukeAction.java @@ -51,6 +51,7 @@ public Set getSegments() return segments; } + @Override public TypeReference getReturnTypeReference() { return new TypeReference() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index d15793e9b95d..806969e860cd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -84,6 +84,7 @@ public DataSourceMetadata getEndMetadata() return endMetadata; } + @Override public TypeReference getReturnTypeReference() { return new TypeReference() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index e3fb7be6f9d7..6a9370324d0a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -31,7 +31,7 @@ public class TaskConfig { public static final List DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of( - "org.apache.hadoop:hadoop-client:2.3.0" + "org.apache.hadoop:hadoop-client:2.7.3" ); private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M"); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index c69381708f22..54b6c5ae308d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -147,6 +147,7 @@ public HadoopIngestionSpec getSpec() return spec; } + @Override @JsonProperty public List getHadoopDependencyCoordinates() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java index 669241a13310..62a7bb06141b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.Injector; - import io.druid.guice.ExtensionsConfig; import io.druid.guice.GuiceInjectors; import io.druid.indexing.common.TaskToolbox; @@ -142,11 +141,14 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo Arrays.asList(((URLClassLoader) HadoopIndexTask.class.getClassLoader()).getURLs()) ); + final List extensionURLs = Lists.newArrayList(); for (final File extension : Initialization.getExtensionFilesToLoad(extensionsConfig)) { final ClassLoader extensionLoader = Initialization.getClassLoaderForExtension(extension); - jobURLs.addAll(Arrays.asList(((URLClassLoader) extensionLoader).getURLs())); + extensionURLs.addAll(Arrays.asList(((URLClassLoader) extensionLoader).getURLs())); } + jobURLs.addAll(extensionURLs); + final List localClassLoaderURLs = new ArrayList<>(jobURLs); // hadoop dependencies come before druid classes because some extensions depend on them @@ -169,11 +171,16 @@ public static ClassLoader buildClassLoader(final List hadoopDependencyCo hadoopContainerDruidClasspathJars = Joiner.on(File.pathSeparator).join(jobURLs); } else { - hadoopContainerDruidClasspathJars = - Joiner.on(File.pathSeparator) - .join( - Initialization.getURLsForClasspath(extensionsConfig.getHadoopContainerDruidClasspath()) - ); + List hadoopContainerURLs = Lists.newArrayList( + Initialization.getURLsForClasspath(extensionsConfig.getHadoopContainerDruidClasspath()) + ); + + if (extensionsConfig.getAddExtensionsToHadoopContainer()) { + hadoopContainerURLs.addAll(extensionURLs); + } + + hadoopContainerDruidClasspathJars = Joiner.on(File.pathSeparator) + .join(hadoopContainerURLs); } log.info("Hadoop Container Druid Classpath is set to [%s]", hadoopContainerDruidClasspathJars); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index e12ee99b458c..743984e32974 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -30,18 +30,21 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; +import com.google.common.util.concurrent.ListenableFuture; import io.druid.common.utils.JodaUtils; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.Rows; import io.druid.guice.annotations.Smile; +import io.druid.hll.HyperLogLogCollector; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskLock; @@ -53,7 +56,8 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.Pair; +import io.druid.java.util.common.granularity.Granularity; +import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.DruidMetrics; @@ -69,20 +73,21 @@ import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.appenderator.Appenderator; import io.druid.segment.realtime.appenderator.AppenderatorConfig; +import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import io.druid.segment.realtime.appenderator.Appenderators; -import io.druid.segment.realtime.appenderator.FiniteAppenderatorDriver; +import io.druid.segment.realtime.appenderator.AppenderatorDriver; import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import io.druid.segment.realtime.firehose.ReplayableFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; import io.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.HashBasedNumberedShardSpec; -import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; +import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; +import org.codehaus.plexus.util.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -92,12 +97,16 @@ import java.io.IOException; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.SortedSet; +import java.util.TreeMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.BiFunction; import java.util.stream.Collectors; +import java.util.stream.IntStream; public class IndexTask extends AbstractTask { @@ -166,28 +175,18 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .bucketIntervals() .isPresent(); - final FirehoseFactory delegateFirehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); - if (delegateFirehoseFactory instanceof IngestSegmentFirehoseFactory) { + if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { // pass toolbox to Firehose - ((IngestSegmentFirehoseFactory) delegateFirehoseFactory).setTaskToolbox(toolbox); + ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); } - final FirehoseFactory firehoseFactory; - if (ingestionSchema.getIOConfig().isSkipFirehoseCaching() - || delegateFirehoseFactory instanceof ReplayableFirehoseFactory) { - firehoseFactory = delegateFirehoseFactory; - } else { - firehoseFactory = new ReplayableFirehoseFactory( - delegateFirehoseFactory, - ingestionSchema.getTuningConfig().isReportParseExceptions(), - null, - null, - smileMapper - ); - } + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + // Firehose temporary directory is automatically removed when this IndexTask completes. + FileUtils.forceMkdir(firehoseTempDir); - final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory); + final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir); final String version; final DataSchema dataSchema; @@ -209,7 +208,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception dataSchema = ingestionSchema.getDataSchema(); } - if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory)) { + if (generateAndPublishSegments(toolbox, dataSchema, shardSpecs, version, firehoseFactory, firehoseTempDir)) { return TaskStatus.success(getId()); } else { return TaskStatus.failure(getId()); @@ -217,136 +216,281 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } /** - * Determines the number of shards for each interval using a hash of queryGranularity timestamp + all dimensions (i.e - * hash-based partitioning). In the future we may want to also support single-dimension partitioning. + * Determines intervals and shardSpecs for input data. This method first checks that it must determine intervals and + * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. + * ShardSpecs must be determined if the perfect rollup must be guaranteed even though the number of shards is not + * specified in {@link IndexTuningConfig}. + *

+ * If both intervals and shardSpecs don't have to be determined, this method simply returns {@link ShardSpecs} for the + * given intervals. Here, if {@link IndexTuningConfig#numShards} is not specified, {@link NumberedShardSpec} is used. + *

+ * If one of intervals or shardSpecs need to be determined, this method reads the entire input for determining one of + * them. If the perfect rollup must be guaranteed, {@link HashBasedNumberedShardSpec} is used for hash partitioning + * of input data. In the future we may want to also support single-dimension partitioning. + * + * @return generated {@link ShardSpecs} representing a map of intervals and corresponding shard specs */ private ShardSpecs determineShardSpecs( final TaskToolbox toolbox, - final FirehoseFactory firehoseFactory + final FirehoseFactory firehoseFactory, + final File firehoseTempDir ) throws IOException { final ObjectMapper jsonMapper = toolbox.getObjectMapper(); - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - final boolean fixedNumPartitions = ingestionSchema.getTuningConfig().getNumShards() != null; - final boolean fixedIntervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .isPresent() - && !ingestionSchema.getTuningConfig().isForceExtendableShardSpecs() - && !ingestionSchema.getIOConfig().isAppendToExisting(); - - final Set intervals; - if (fixedIntervals) { - log.info("intervals provided, skipping determine partition scan"); - intervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .get(); - } else { - // determine intervals containing data - log.info("Determining intervals"); - intervals = new HashSet<>(); - long determineIntervalsStartMillis = System.currentTimeMillis(); + final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); + final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); - try (final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser())) { - while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); - final Interval interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + final Granularity queryGranularity = granularitySpec.getQueryGranularity(); + + final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent(); + // Guarenteed rollup means that this index task guarantees the 'perfect rollup' across the entire data set. + final boolean guaranteedRollup = tuningConfig.isForceGuaranteedRollup() && + !tuningConfig.isForceExtendableShardSpecs() && + !ioConfig.isAppendToExisting(); + final boolean determineNumPartitions = tuningConfig.getNumShards() == null && guaranteedRollup; + final boolean useExtendableShardSpec = !guaranteedRollup; + + // if we were given number of shards per interval and the intervals, we don't need to scan the data + if (!determineNumPartitions && !determineIntervals) { + log.info("Skipping determine partition scan"); + + final int numShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); + final BiFunction shardSpecCreateFn = getShardSpecCreateFunction( + useExtendableShardSpec, + numShards, + jsonMapper + ); - intervals.add(interval); - } + final Map> intervalToShardSpecs = new HashMap<>(); + for (Interval interval : granularitySpec.bucketIntervals().get()) { + final List intervalShardSpecs = IntStream.range(0, numShards) + .mapToObj( + shardId -> shardSpecCreateFn.apply(shardId, numShards) + ) + .collect(Collectors.toList()); + intervalToShardSpecs.put(interval, intervalShardSpecs); } - log.info("Found intervals in %,dms", System.currentTimeMillis() - determineIntervalsStartMillis); + if (useExtendableShardSpec) { + return createExtendableShardSpecs(intervalToShardSpecs); + } else { + return createNonExtendableShardSpecs(intervalToShardSpecs); + } } - if (fixedNumPartitions) { - final int numShards = ingestionSchema.getTuningConfig().getNumShards(); - final Map> intervalToShardSpecs = new HashMap<>(); + // determine intervals containing data and prime HLL collectors + final Map> hllCollectors = new TreeMap<>( + Comparators.intervalsByStartThenEnd() + ); + int thrownAway = 0; - for (Interval interval : intervals) { - final List intervalShardSpecs = Lists.newArrayListWithCapacity(numShards); - if (numShards > 1) { - for (int i = 0; i < numShards; i++) { - intervalShardSpecs.add(new HashBasedNumberedShardSpec(i, numShards, null, jsonMapper)); + log.info("Determining intervals and shardSpecs"); + long determineShardSpecsStartMillis = System.currentTimeMillis(); + try ( + final Firehose firehose = firehoseFactory.connect( + ingestionSchema.getDataSchema().getParser(), + firehoseTempDir + ) + ) { + while (firehose.hasMore()) { + final InputRow inputRow = firehose.nextRow(); + + // The null inputRow means the caller must skip this row. + if (inputRow == null) { + continue; + } + + final Interval interval; + if (determineIntervals) { + interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); + } else { + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!optInterval.isPresent()) { + thrownAway++; + continue; + } + interval = optInterval.get(); + } + + if (determineNumPartitions) { + if (!hllCollectors.containsKey(interval)) { + hllCollectors.put(interval, Optional.of(HyperLogLogCollector.makeLatestCollector())); } + + List groupKey = Rows.toGroupKey( + queryGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), + inputRow + ); + hllCollectors.get(interval).get() + .add(hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asBytes()); } else { - intervalShardSpecs.add(NoneShardSpec.instance()); + // we don't need to determine partitions but we still need to determine intervals, so add an Optional.absent() + // for the interval and don't instantiate a HLL collector + if (!hllCollectors.containsKey(interval)) { + hllCollectors.put(interval, Optional.absent()); + } } - intervalToShardSpecs.put(interval, intervalShardSpecs); } + } - return new ShardSpecs() - { + if (thrownAway > 0) { + log.warn("Unable to to find a matching interval for [%,d] events", thrownAway); + } - @Override - public Collection getIntervals() - { - return intervalToShardSpecs.keySet(); - } + final Map> intervalToShardSpecs = new HashMap<>(); + final int defaultNumShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); + for (final Map.Entry> entry : hllCollectors.entrySet()) { + final Interval interval = entry.getKey(); + final Optional collector = entry.getValue(); + + final int numShards; + if (determineNumPartitions) { + final long numRows = new Double(collector.get().estimateCardinality()).longValue(); + numShards = (int) Math.ceil((double) numRows / tuningConfig.getTargetPartitionSize()); + log.info("Estimated [%,d] rows of data for interval [%s], creating [%,d] shards", numRows, interval, numShards); + } else { + numShards = defaultNumShards; + log.info("Creating [%,d] shards for interval [%s]", numShards, interval); + } - @Override - public boolean isExtendable() - { - return false; - } + final BiFunction shardSpecCreateFn = getShardSpecCreateFunction( + useExtendableShardSpec, + numShards, + jsonMapper + ); - @Override - public ShardSpec getShardSpec(Interval interval, InputRow row) - { - final List shardSpecs = intervalToShardSpecs.get(interval); - if (shardSpecs == null || shardSpecs.isEmpty()) { - throw new ISE("Failed to get shardSpec for interval[%s]", interval); - } - return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(row.getTimestampFromEpoch(), row); - } + final List intervalShardSpecs = IntStream.range(0, numShards) + .mapToObj( + shardId -> shardSpecCreateFn.apply(shardId, numShards) + ).collect(Collectors.toList()); - @Override - public void updateShardSpec(Interval interval) - { - // do nothing - } - }; + intervalToShardSpecs.put(interval, intervalShardSpecs); + } + log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); + + if (useExtendableShardSpec) { + return createExtendableShardSpecs(intervalToShardSpecs); } else { - final Map shardSpecMap = intervals.stream() - .collect(Collectors.toMap(interval -> interval, key -> new LinearShardSpec(0))); - return new ShardSpecs() + return createNonExtendableShardSpecs(intervalToShardSpecs); + } + } + + private static ShardSpecs createNonExtendableShardSpecs(Map> intervalToShardSpecs) + { + return new ShardSpecs() + { + @Override + public Collection getIntervals() { - @Override - public Collection getIntervals() - { - return shardSpecMap.keySet(); - } + return intervalToShardSpecs.keySet(); + } - @Override - public boolean isExtendable() - { - return true; + @Override + public ShardSpec getShardSpec(Interval interval, InputRow row) + { + final List shardSpecs = intervalToShardSpecs.get(interval); + if (shardSpecs == null || shardSpecs.isEmpty()) { + throw new ISE("Failed to get shardSpec for interval[%s]", interval); } + return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(row.getTimestampFromEpoch(), row); + } - @Override - public ShardSpec getShardSpec(Interval interval, InputRow row) - { - return shardSpecMap.get(interval); - } + @Override + public void updateShardSpec(Interval interval) + { + // do nothing + } + }; + } - @Override - public void updateShardSpec(Interval interval) - { - final LinearShardSpec previous = shardSpecMap.get(interval); - Preconditions.checkNotNull(previous, "previous shardSpec for interval[%s]", interval); - shardSpecMap.put(interval, new LinearShardSpec(previous.getPartitionNum() + 1)); - } - }; + private static ShardSpecs createExtendableShardSpecs(Map> intervalToShardSpec) + { + return new ShardSpecs() + { + private final Map shardSpecMap = new HashMap<>(intervalToShardSpec.size()); + + { + intervalToShardSpec.forEach((interval, shardSpecs) -> { + Preconditions.checkState(shardSpecs.size() == 1); + shardSpecMap.put(interval, shardSpecs.get(0)); + }); + } + + @Override + public Collection getIntervals() + { + return shardSpecMap.keySet(); + } + + @Override + public ShardSpec getShardSpec(Interval interval, InputRow row) + { + return shardSpecMap.get(interval); + } + + @Override + public void updateShardSpec(Interval interval) + { + final ShardSpec shardSpec = shardSpecMap.get(interval); + Preconditions.checkState( + shardSpec instanceof NumberedShardSpec, + "shardSpec[%s] must be NumberedShardSpec", + shardSpec.getClass().getCanonicalName() + ); + final NumberedShardSpec previous = (NumberedShardSpec) shardSpec; + Preconditions.checkNotNull(previous, "previous shardSpec for interval[%s] is null", interval); + shardSpecMap.put(interval, new NumberedShardSpec(previous.getPartitionNum() + 1, previous.getPartitions())); + } + }; + } + + private static BiFunction getShardSpecCreateFunction( + boolean useExtendableShardSpec, + Integer numShards, + ObjectMapper jsonMapper + ) + { + if (useExtendableShardSpec) { + return NumberedShardSpec::new; + } else { + if (numShards == null) { + throw new ISE("numShards must not be null"); + } + + if (numShards == 1) { + return (shardId, totalNumShards) -> NoneShardSpec.instance(); + } else { + return (shardId, totalNumShards) -> new HashBasedNumberedShardSpec(shardId, totalNumShards, null, jsonMapper); + } } } + /** + * This method reads input data row by row and adds the read row to a proper segment using {@link AppenderatorDriver}. + * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs + * if one of below conditions are satisfied. + * + *
    + *
  • + * If the number of rows in a segment exceeds {@link IndexTuningConfig#targetPartitionSize} + *
  • + *
  • + * If the number of rows added to {@link AppenderatorDriver} so far exceeds {@link IndexTuningConfig#maxTotalRows} + *
  • + *
+ * + * At the end of this method, all the remaining segments are published. + * + * @return true if generated segments are successfully published, otherwise false + */ private boolean generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, final ShardSpecs shardSpecs, final String version, - final FirehoseFactory firehoseFactory + final FirehoseFactory firehoseFactory, + final File firehoseTempDir ) throws IOException, InterruptedException { @@ -365,41 +509,51 @@ dataSchema, new RealtimeIOConfig(null, null, null), null ); } + final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); + final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig; + final long publishTimeout = tuningConfig.getPublishTimeout(); + final long maxRowsInAppenderator = tuningConfig.getMaxTotalRows(); + final int maxRowsInSegment = tuningConfig.getTargetPartitionSize() == null + ? Integer.MAX_VALUE + : tuningConfig.getTargetPartitionSize(); + final SegmentAllocator segmentAllocator; - if (ingestionSchema.getIOConfig().isAppendToExisting()) { + if (ioConfig.isAppendToExisting()) { segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); } else { - segmentAllocator = (timestamp, row, sequenceName, previousSegmentId) -> { - Optional interval = granularitySpec.bucketInterval(timestamp); - if (!interval.isPresent()) { + segmentAllocator = (row, sequenceName, previousSegmentId) -> { + final DateTime timestamp = row.getTimestamp(); + Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { throw new ISE("Could not find interval for timestamp [%s]", timestamp); } - ShardSpec shardSpec = shardSpecs.getShardSpec(interval.get(), row); + final Interval interval = maybeInterval.get(); + final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, row); if (shardSpec == null) { - throw new ISE("Could not find ShardSpec for sequenceName [%s]", sequenceName); + throw new ISE("Could not find shardSpec for interval[%s]", interval); } - return new SegmentIdentifier(getDataSource(), interval.get(), version, shardSpec); + return new SegmentIdentifier(getDataSource(), interval, version, shardSpec); }; } + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments, null, null); + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + try ( - final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema); - final FiniteAppenderatorDriver driver = newDriver( + final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); + final AppenderatorDriver driver = newDriver( appenderator, toolbox, segmentAllocator, - fireDepartmentMetrics, - ingestionSchema.getTuningConfig() + fireDepartmentMetrics ); - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser()) + final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { final Supplier committerSupplier = Committers.supplierFromFirehose(firehose); - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments, null, null); - return toolbox.getTaskActionClient().submit(action).isSuccess(); - }; if (driver.startJob() != null) { driver.clear(); @@ -410,6 +564,10 @@ dataSchema, new RealtimeIOConfig(null, null, null), null try { final InputRow inputRow = firehose.nextRow(); + if (inputRow == null) { + continue; + } + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); if (!optInterval.isPresent()) { fireDepartmentMetrics.incrementThrownAway(); @@ -417,24 +575,38 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } final Interval interval = optInterval.get(); - final String sequenceName = Appenderators.getSequenceName( - interval, version, shardSpecs.getShardSpec(interval, inputRow) - ); - final Pair> pair = driver.add( - inputRow, sequenceName, committerSupplier, publisher, shardSpecs.isExtendable() - ); - final SegmentIdentifier identifier = pair.lhs; - final List publishedSegments = pair.rhs; - - if (identifier == null) { - throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); + final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, inputRow); + final String sequenceName = Appenderators.getSequenceName(interval, version, shardSpec); + final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); + + if (addResult.isOk()) { + if (addResult.getNumRowsInSegment() >= maxRowsInSegment || + addResult.getTotalNumRowsInAppenderator() >= maxRowsInAppenderator) { + // There can be some segments waiting for being published even though any rows won't be added to them. + // If those segments are not published here, the available space in appenderator will be kept to be small + // which makes the size of segments smaller. + final SegmentsAndMetadata published = awaitPublish( + driver.publishAll( + publisher, + committerSupplier.get() + ), + publishTimeout + ); + published.getSegments().forEach(segment -> shardSpecs.updateShardSpec(segment.getInterval())); + // Even though IndexTask uses NoopHandoffNotifier which does nothing for segment handoff, + // the below code is needed to update the total number of rows added to the appenderator so far. + // See AppenderatorDriver.registerHandoff() and Appenderator.drop(). + // A hard-coded timeout is used here because the below get() is expected to return immediately. + driver.registerHandoff(published).get(1000, TimeUnit.MILLISECONDS); + } + } else { + throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); } - publishedSegments.forEach(segmentId -> shardSpecs.updateShardSpec(segmentId.getInterval())); fireDepartmentMetrics.incrementProcessed(); } catch (ParseException e) { - if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { + if (tuningConfig.isReportParseExceptions()) { throw e; } else { fireDepartmentMetrics.incrementUnparseable(); @@ -446,7 +618,14 @@ dataSchema, new RealtimeIOConfig(null, null, null), null driver.persist(committerSupplier.get()); } - final SegmentsAndMetadata published = driver.publishAndWaitHandoff(publisher, committerSupplier.get()); + final SegmentsAndMetadata published = awaitPublish( + driver.publishAll( + publisher, + committerSupplier.get() + ), + publishTimeout + ); + if (published == null) { log.error("Failed to publish segments, aborting!"); return false; @@ -469,40 +648,55 @@ public String apply(DataSegment input) return true; } } + catch (TimeoutException | ExecutionException e) { + throw Throwables.propagate(e); + } + } + + private static SegmentsAndMetadata awaitPublish( + ListenableFuture publishFuture, + long publishTimeout + ) + throws ExecutionException, InterruptedException, TimeoutException + { + if (publishTimeout == 0) { + return publishFuture.get(); + } else { + return publishFuture.get(publishTimeout, TimeUnit.MILLISECONDS); + } } - private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox, DataSchema dataSchema) + private static Appenderator newAppenderator( + FireDepartmentMetrics metrics, + TaskToolbox toolbox, + DataSchema dataSchema, + IndexTuningConfig tuningConfig + ) { return Appenderators.createOffline( dataSchema, - ingestionSchema.getTuningConfig().withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")), + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, toolbox.getSegmentPusher(), toolbox.getObjectMapper(), toolbox.getIndexIO(), - ingestionSchema.getTuningConfig().isBuildV9Directly() ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger() + tuningConfig.isBuildV9Directly() ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger() ); } - private FiniteAppenderatorDriver newDriver( + private static AppenderatorDriver newDriver( final Appenderator appenderator, final TaskToolbox toolbox, final SegmentAllocator segmentAllocator, - final FireDepartmentMetrics metrics, - final IndexTuningConfig tuningConfig + final FireDepartmentMetrics metrics ) { - return new FiniteAppenderatorDriver( + return new AppenderatorDriver( appenderator, segmentAllocator, new NoopSegmentHandoffNotifierFactory(), // don't wait for handoff since we don't serve queries new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), - // If targetPartitionSize is null, numShards must be set which means intervals are already partitioned with - // proper shardSpecs. See determineShardSpecs(). - tuningConfig.getTargetPartitionSize() == null ? Integer.MAX_VALUE : tuningConfig.getTargetPartitionSize(), - tuningConfig.getMaxPersistedSegmentsBytes(), - 0, metrics ); } @@ -520,13 +714,6 @@ private interface ShardSpecs */ Collection getIntervals(); - /** - * Indicate that the type of shardSpecs is extendable like {@link LinearShardSpec}. - * - * @return true if the type of shardSpecs is extendable - */ - boolean isExtendable(); - /** * Return a shardSpec for the given interval and input row. * @@ -538,7 +725,7 @@ private interface ShardSpecs /** * Update the shardSpec of the given interval. When the type of shardSpecs is extendable, this method must update - * the shardSpec properly. For example, if the {@link LinearShardSpec} is used, an implementation of this method + * the shardSpec properly. For example, if the {@link NumberedShardSpec} is used, an implementation of this method * may replace the shardSpec of the given interval with a new one having a greater partitionNum. * * @param interval interval for shardSpec to be updated @@ -563,10 +750,12 @@ public IndexIngestionSpec( this.dataSchema = dataSchema; this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig == null - ? - new IndexTuningConfig(null, null, null, null, null, null, null, null, null, (File) null) - : tuningConfig; + this.tuningConfig = tuningConfig == null ? new IndexTuningConfig() : tuningConfig; + + if (this.ioConfig.isAppendToExisting() && this.tuningConfig.isForceGuaranteedRollup()) { + log.warn("Perfect rollup is not guaranteed when appending to existing data sources." + + " forceGuaranteedRollup flag is ignored."); + } } @Override @@ -595,22 +784,18 @@ public IndexTuningConfig getTuningConfig() public static class IndexIOConfig implements IOConfig { private static final boolean DEFAULT_APPEND_TO_EXISTING = false; - private static final boolean DEFAULT_SKIP_FIREHOSE_CACHING = false; private final FirehoseFactory firehoseFactory; private final boolean appendToExisting; - private final boolean skipFirehoseCaching; @JsonCreator public IndexIOConfig( @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting, - @JsonProperty("skipFirehoseCaching") @Nullable Boolean skipFirehoseCaching + @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) { this.firehoseFactory = firehoseFactory; this.appendToExisting = appendToExisting == null ? DEFAULT_APPEND_TO_EXISTING : appendToExisting; - this.skipFirehoseCaching = skipFirehoseCaching == null ? DEFAULT_SKIP_FIREHOSE_CACHING : skipFirehoseCaching; } @JsonProperty("firehose") @@ -624,76 +809,85 @@ public boolean isAppendToExisting() { return appendToExisting; } - - @JsonProperty("skipFirehoseCaching") - public boolean isSkipFirehoseCaching() - { - return skipFirehoseCaching; - } } @JsonTypeName("index") public static class IndexTuningConfig implements TuningConfig, AppenderatorConfig { - private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75000; - private static final long DEFAULT_MAX_PERSISTED_SEGMENTS_BYTES = 1024 * 1024 * 1024; + private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; + private static final int DEFAULT_MAX_TOTAL_ROWS = 150_000; private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final boolean DEFAULT_BUILD_V9_DIRECTLY = true; private static final boolean DEFAULT_FORCE_EXTENDABLE_SHARD_SPECS = false; + private static final boolean DEFAULT_GUARANTEE_ROLLUP = false; private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; + private static final long DEFAULT_PUBLISH_TIMEOUT = 0; static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000; private final Integer targetPartitionSize; private final int maxRowsInMemory; - private final long maxPersistedSegmentsBytes; + private final int maxTotalRows; private final Integer numShards; private final IndexSpec indexSpec; private final File basePersistDirectory; private final int maxPendingPersists; private final boolean buildV9Directly; private final boolean forceExtendableShardSpecs; + private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; + private final long publishTimeout; @JsonCreator public IndexTuningConfig( @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, - @JsonProperty("maxPersistedSegmentsBytes") @Nullable Long maxPersistedSegmentsBytes, + @JsonProperty("maxTotalRows") @Nullable Integer maxTotalRows, @JsonProperty("rowFlushBoundary") @Nullable Integer rowFlushBoundary_forBackCompatibility, // DEPRECATED @JsonProperty("numShards") @Nullable Integer numShards, @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, - @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions + @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, + @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("publishTimeout") @Nullable Long publishTimeout ) { this( targetPartitionSize, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, - maxPersistedSegmentsBytes, + maxTotalRows, numShards, indexSpec, maxPendingPersists, buildV9Directly, forceExtendableShardSpecs, + forceGuaranteedRollup, reportParseExceptions, + publishTimeout, null ); } + private IndexTuningConfig() + { + this(null, null, null, null, null, null, null, null, null, null, null, (File) null); + } + private IndexTuningConfig( @Nullable Integer targetPartitionSize, @Nullable Integer maxRowsInMemory, - @Nullable Long maxPersistedSegmentsBytes, + @Nullable Integer maxTotalRows, @Nullable Integer numShards, @Nullable IndexSpec indexSpec, @Nullable Integer maxPendingPersists, @Nullable Boolean buildV9Directly, @Nullable Boolean forceExtendableShardSpecs, + @Nullable Boolean forceGuaranteedRollup, @Nullable Boolean reportParseExceptions, + @Nullable Long publishTimeout, @Nullable File basePersistDirectory ) { @@ -708,9 +902,9 @@ private IndexTuningConfig( ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize); this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - this.maxPersistedSegmentsBytes = maxPersistedSegmentsBytes == null - ? DEFAULT_MAX_PERSISTED_SEGMENTS_BYTES - : maxPersistedSegmentsBytes; + this.maxTotalRows = maxTotalRows == null + ? DEFAULT_MAX_TOTAL_ROWS + : maxTotalRows; this.numShards = numShards == null || numShards.equals(-1) ? null : numShards; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; @@ -718,10 +912,16 @@ private IndexTuningConfig( this.forceExtendableShardSpecs = forceExtendableShardSpecs == null ? DEFAULT_FORCE_EXTENDABLE_SHARD_SPECS : forceExtendableShardSpecs; + this.forceGuaranteedRollup = forceGuaranteedRollup == null ? DEFAULT_GUARANTEE_ROLLUP : forceGuaranteedRollup; this.reportParseExceptions = reportParseExceptions == null ? DEFAULT_REPORT_PARSE_EXCEPTIONS : reportParseExceptions; + this.publishTimeout = publishTimeout == null ? DEFAULT_PUBLISH_TIMEOUT : publishTimeout; this.basePersistDirectory = basePersistDirectory; + + if (this.forceGuaranteedRollup && this.forceExtendableShardSpecs) { + log.warn("Perfect rollup is not guaranteed with extendable shardSpecs. forceGuaranteedRollup flag is ignored."); + } } public IndexTuningConfig withBasePersistDirectory(File dir) @@ -729,13 +929,15 @@ public IndexTuningConfig withBasePersistDirectory(File dir) return new IndexTuningConfig( targetPartitionSize, maxRowsInMemory, - maxPersistedSegmentsBytes, + maxTotalRows, numShards, indexSpec, maxPendingPersists, buildV9Directly, forceExtendableShardSpecs, + forceGuaranteedRollup, reportParseExceptions, + publishTimeout, dir ); } @@ -754,10 +956,9 @@ public int getMaxRowsInMemory() } @JsonProperty - @Override - public long getMaxPersistedSegmentsBytes() + public int getMaxTotalRows() { - return maxPersistedSegmentsBytes; + return maxTotalRows; } @JsonProperty @@ -792,6 +993,18 @@ public boolean isBuildV9Directly() return buildV9Directly; } + @JsonProperty + public boolean isForceExtendableShardSpecs() + { + return forceExtendableShardSpecs; + } + + @JsonProperty + public boolean isForceGuaranteedRollup() + { + return forceGuaranteedRollup; + } + @JsonProperty @Override public boolean isReportParseExceptions() @@ -800,9 +1013,9 @@ public boolean isReportParseExceptions() } @JsonProperty - public boolean isForceExtendableShardSpecs() + public long getPublishTimeout() { - return forceExtendableShardSpecs; + return publishTimeout; } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index fdec7e63c1b3..de49180bf08b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -130,7 +130,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception final ServiceEmitter emitter = toolbox.getEmitter(); final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); - final File taskDir = toolbox.getTaskWorkDir(); + final File mergeDir = toolbox.getMergeDir(); try { final long startTime = System.currentTimeMillis(); @@ -155,7 +155,7 @@ public String apply(DataSegment input) final Map gettedSegments = toolbox.fetchSegments(segments); // merge files together - final File fileToUpload = merge(toolbox, gettedSegments, new File(taskDir, "merged")); + final File fileToUpload = merge(toolbox, gettedSegments, mergeDir); emitter.emit(builder.build("merger/numMerged", segments.size())); emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java index 0eac7c88ec2a..aa041f09e335 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; - import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.indexing.common.TaskStatus; @@ -30,7 +29,6 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; - import org.joda.time.DateTime; import java.util.Map; @@ -139,7 +137,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception if (firehoseFactory != null) { log.info("Connecting firehose"); } - try (Firehose firehose = firehoseFactory != null ? firehoseFactory.connect(null) : null) { + try (Firehose firehose = firehoseFactory != null ? firehoseFactory.connect(null, null) : null) { log.info("Running noop task[%s]", getId()); log.info("Sleeping for %,d millis.", runTime); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 60bf62a9d3af..1e5724f8b8e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -63,6 +63,7 @@ import io.druid.segment.realtime.plumber.VersioningPolicy; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -248,12 +249,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } } } - - @Override - public boolean isAnnounced(DataSegment segment) - { - return toolbox.getSegmentAnnouncer().isAnnounced(segment); - } }; // NOTE: getVersion will block if there is lock contention, which will block plumber.getSink @@ -283,7 +278,7 @@ public String getVersion(final Interval interval) DataSchema dataSchema = spec.getDataSchema(); RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); RealtimeTuningConfig tuningConfig = spec.getTuningConfig() - .withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")) + .withBasePersistDirectory(toolbox.getPersistDir()) .withVersioningPolicy(versioningPolicy); final FireDepartment fireDepartment = new FireDepartment( @@ -324,13 +319,19 @@ public String getVersion(final Interval interval) this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); Supplier committerSupplier = null; + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); try { + toolbox.getDataSegmentServerAnnouncer().announce(); + plumber.startJob(); // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); + // Firehose temporary directory is automatically removed when this RealtimeIndexTask completes. + FileUtils.forceMkdir(firehoseTempDir); + // Delay firehose connection to avoid claiming input resources while the plumber is starting up. final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); @@ -338,7 +339,7 @@ public String getVersion(final Interval interval) // Skip connecting firehose if we've been stopped before we got started. synchronized (this) { if (!gracefullyStopped) { - firehose = firehoseFactory.connect(spec.getDataSchema().getParser()); + firehose = firehoseFactory.connect(spec.getDataSchema().getParser(), firehoseTempDir); committerSupplier = Committers.supplierFromFirehose(firehose); } } @@ -425,6 +426,8 @@ public void run() toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); } } + + toolbox.getDataSegmentServerAnnouncer().unannounce(); } log.info("Job done!"); diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 25028f0f5c0f..11bc9bdfa26c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -127,7 +127,7 @@ public void setTaskToolbox(TaskToolbox taskToolbox) } @Override - public Firehose connect(InputRowParser inputRowParser) throws IOException, ParseException + public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws IOException, ParseException { log.info("Connecting firehose: dataSource[%s], interval[%s]", dataSource, interval); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 6ab7e39066e9..b68181242902 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -516,6 +516,7 @@ public TaskStatus call() } } + @Override @LifecycleStop public void stop() { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableWorkerInfo.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableWorkerInfo.java index 9d4026004f7e..67ac29671495 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableWorkerInfo.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ImmutableWorkerInfo.java @@ -74,6 +74,10 @@ public Set getAvailabilityGroups() return availabilityGroups; } + public int getAvailableCapacity() { + return getWorker().getCapacity() - getCurrCapacityUsed(); + } + @JsonProperty("runningTasks") public Set getRunningTasks() { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index e019465fc7b9..fb66e53e610c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -310,8 +310,11 @@ public void onFailure(Throwable throwable) waitingFor.decrement(); waitingForMonitor.notifyAll(); } - default: break; + case CONNECTION_SUSPENDED: + case CONNECTION_RECONNECTED: + case CONNECTION_LOST: + // do nothing } } } @@ -425,12 +428,14 @@ public Collection getPendingTasks() return ImmutableList.copyOf(pendingTasks.values()); } + @Override public Collection getPendingTaskPayloads() { // return a snapshot of current pending task payloads. return ImmutableList.copyOf(pendingTaskPayloads.values()); } + @Override public RemoteTaskRunnerConfig getConfig() { return config; @@ -1001,6 +1006,11 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th retVal.setException(new IllegalStateException(message)); } runPendingTasks(); + break; + case CONNECTION_SUSPENDED: + case CONNECTION_RECONNECTED: + case CONNECTION_LOST: + // do nothing } } catch (Exception e) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index e70a24852452..01213fc0ee72 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -242,12 +242,12 @@ private void stopLeading() } } - public boolean isLeading() + public boolean isLeader() { return leading; } - public String getLeader() + public String getCurrentLeader() { try { final Participant leader = leaderSelector.getLeader(); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerResourceManagementConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerResourceManagementConfig.java index 4d107865797c..7d120ed5c4d3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerResourceManagementConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerResourceManagementConfig.java @@ -41,30 +41,35 @@ public PendingTaskBasedWorkerResourceManagementConfig setMaxScalingStep(int maxS return this; } + @Override public PendingTaskBasedWorkerResourceManagementConfig setWorkerIdleTimeout(Period workerIdleTimeout) { super.setWorkerIdleTimeout(workerIdleTimeout); return this; } + @Override public PendingTaskBasedWorkerResourceManagementConfig setMaxScalingDuration(Period maxScalingDuration) { super.setMaxScalingDuration(maxScalingDuration); return this; } + @Override public PendingTaskBasedWorkerResourceManagementConfig setNumEventsToTrack(int numEventsToTrack) { super.setNumEventsToTrack(numEventsToTrack); return this; } + @Override public PendingTaskBasedWorkerResourceManagementConfig setWorkerVersion(String workerVersion) { super.setWorkerVersion(workerVersion); return this; } + @Override public PendingTaskBasedWorkerResourceManagementConfig setPendingTaskTimeout(Period pendingTaskTimeout) { super.setPendingTaskTimeout(pendingTaskTimeout); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerResourceManagementStrategy.java index 1a7696565ea3..4907ac0baaa3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/SimpleWorkerResourceManagementStrategy.java @@ -94,6 +94,7 @@ public SimpleWorkerResourceManagementStrategy( } + @Override protected boolean doProvision(WorkerTaskRunner runner) { Collection pendingTasks = runner.getPendingTasks(); @@ -163,6 +164,7 @@ public String apply(ImmutableWorkerInfo input) } } + @Override boolean doTerminate(WorkerTaskRunner runner) { Collection pendingTasks = runner.getPendingTasks(); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java index d02e294fed4d..7ea9597334a9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java @@ -20,17 +20,24 @@ package io.druid.indexing.overlord.http; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.druid.indexing.overlord.TaskMaster; import io.druid.server.http.RedirectInfo; import java.net.URI; import java.net.URL; +import java.util.Set; /** */ public class OverlordRedirectInfo implements RedirectInfo { + private static final Set LOCAL_PATHS = ImmutableSet.of( + "/druid/indexer/v1/leader", + "/druid/indexer/v1/isLeader" + ); + private final TaskMaster taskMaster; @Inject @@ -42,14 +49,14 @@ public OverlordRedirectInfo(TaskMaster taskMaster) @Override public boolean doLocal(String requestURI) { - return taskMaster.isLeading(); + return (requestURI != null && LOCAL_PATHS.contains(requestURI)) || taskMaster.isLeader(); } @Override public URL getRedirectURL(String queryString, String requestURI) { try { - final String leader = taskMaster.getLeader(); + final String leader = taskMaster.getCurrentLeader(); if (leader == null || leader.isEmpty()) { return null; } else { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 7d5775b8be1a..9683d5dcf5d9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -177,7 +177,22 @@ public Response apply(TaskQueue taskQueue) @Produces(MediaType.APPLICATION_JSON) public Response getLeader() { - return Response.ok(taskMaster.getLeader()).build(); + return Response.ok(taskMaster.getCurrentLeader()).build(); + } + + @GET + @Path("/isLeader") + @ResourceFilters(StateResourceFilter.class) + @Produces(MediaType.APPLICATION_JSON) + public Response isLeader() + { + final boolean leading = taskMaster.isLeader(); + final Map response = ImmutableMap.of("leader", leading); + if (leading) { + return Response.ok(response).build(); + } else { + return Response.status(Response.Status.NOT_FOUND).entity(response).build(); + } } @GET diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategy.java index ef0a41c2f63e..becc8053b54b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategy.java @@ -22,7 +22,6 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.ImmutableWorkerInfo; import io.druid.indexing.overlord.config.WorkerTaskRunnerConfig; @@ -39,28 +38,13 @@ public Optional findWorkerForTask( WorkerTaskRunnerConfig config, ImmutableMap zkWorkers, Task task ) { + // the version sorting is needed because if the workers have the same available capacity only one of them is + // returned. Exists the possibility that this worker is disabled and doesn't have valid version so can't + // run new tasks, so in this case the workers are sorted using version to ensure that if exists enable + // workers the comparator return one of them. final TreeSet sortedWorkers = Sets.newTreeSet( - new Comparator() - { - @Override - public int compare( - ImmutableWorkerInfo zkWorker, ImmutableWorkerInfo zkWorker2 - ) - { - int retVal = -Ints.compare(zkWorker2.getCurrCapacityUsed(), zkWorker.getCurrCapacityUsed()); - // the version sorting is needed because if the workers have the same currCapacityUsed only one of them is - // returned. Exists the possibility that this worker is disabled and doesn't have valid version so can't - // run new tasks, so in this case the workers are sorted using version to ensure that if exists enable - // workers the comparator return one of them. - - if(retVal == 0) { - retVal = zkWorker2.getWorker().getVersion().compareTo(zkWorker.getWorker().getVersion()); - } - - return retVal; - } - } - ); + Comparator.comparing(ImmutableWorkerInfo::getAvailableCapacity).reversed() + .thenComparing(zkWorker -> zkWorker.getWorker().getVersion())); sortedWorkers.addAll(zkWorkers.values()); final String minWorkerVer = config.getMinWorkerVersion(); diff --git a/indexing-service/src/main/resources/indexer_static/console.html b/indexing-service/src/main/resources/indexer_static/console.html index a9a5506b256e..d397b1bbbea8 100644 --- a/indexing-service/src/main/resources/indexer_static/console.html +++ b/indexing-service/src/main/resources/indexer_static/console.html @@ -37,6 +37,11 @@
Coordinator Console
+ +

Running Tasks

Loading Running Tasks... this may take a few minutes
diff --git a/indexing-service/src/main/resources/indexer_static/js/console-0.0.1.js b/indexing-service/src/main/resources/indexer_static/js/console-0.0.1.js index a8eb8bc7cead..5ec87c588fa4 100644 --- a/indexing-service/src/main/resources/indexer_static/js/console-0.0.1.js +++ b/indexing-service/src/main/resources/indexer_static/js/console-0.0.1.js @@ -16,6 +16,34 @@ var killTask = function(taskId) { } } +var resetSupervisor = function(supervisorId) { + if(confirm('Do you really want to reset: '+ supervisorId)) { + $.ajax({ + type:'POST', + url: '/druid/indexer/v1/supervisor/' + supervisorId + '/reset', + data: '' + }).done(function(data) { + setTimeout(function() { location.reload(true) }, 750); + }).fail(function(data) { + alert('Reset request failed, please check overlord logs for details.'); + }) + } +} + +var shutdownSupervisor = function(supervisorId) { + if(confirm('Do you really want to shutdown: '+ supervisorId)) { + $.ajax({ + type:'POST', + url: '/druid/indexer/v1/supervisor/' + supervisorId + '/shutdown', + data: '' + }).done(function(data) { + setTimeout(function() { location.reload(true) }, 750); + }).fail(function(data) { + alert('Shutdown request failed, please check overlord logs for details.'); + }) + } +} + $(document).ready(function() { var augment = function(data, showKill) { for (i = 0 ; i < data.length ; i++) { @@ -31,6 +59,26 @@ $(document).ready(function() { } } + $.get('/druid/indexer/v1/supervisor', function(dataList) { + var data = [] + for (i = 0 ; i < dataList.length ; i++) { + var supervisorId = encodeURIComponent(dataList[i]) + data[i] = { + "dataSource" : supervisorId, + "more" : + 'payload' + + 'status' + + 'history' + + 'reset' + + 'shutdown' + } + } + buildTable((data), $('#supervisorsTable')); + if (dataList.length > 0) { + $('.supervisors_section').show(); + } + }); + $.get('/druid/indexer/v1/runningTasks', function(data) { $('.running_loading').hide(); augment(data, true); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java index 86a513816ab2..76e23a4f393f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java @@ -41,6 +41,7 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -102,6 +103,7 @@ public void setUp() throws IOException mockDataSegmentMover, mockDataSegmentArchiver, mockSegmentAnnouncer, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), mockHandoffNotifierFactory, mockQueryRunnerFactoryConglomerate, mockQueryExecutorService, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java index 6f3034a47933..50d756086317 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/RemoteTaskActionClientTest.java @@ -19,11 +19,13 @@ package io.druid.indexing.common.actions; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; - +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; import io.druid.client.selector.Server; import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.indexing.common.RetryPolicyConfig; @@ -32,13 +34,6 @@ import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.jackson.DefaultObjectMapper; - -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import org.easymock.EasyMock; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Interval; @@ -46,13 +41,16 @@ import org.junit.Before; import org.junit.Test; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.util.concurrent.Futures; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.Request; -import com.metamx.http.client.response.StatusResponseHandler; -import com.metamx.http.client.response.StatusResponseHolder; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; public class RemoteTaskActionClientTest { @@ -99,7 +97,12 @@ public String getAddress() long now = System.currentTimeMillis(); - result = Arrays.asList(new TaskLock("groupId", "dataSource", new Interval(now - 30 * 1000, now), "version")); + result = Collections.singletonList(new TaskLock( + "groupId", + "dataSource", + new Interval(now - 30 * 1000, now), + "version" + )); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 0fe96baccdce..be8080b82140 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.Lists; import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; @@ -51,7 +52,7 @@ import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.LinearShardSpec; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; @@ -65,6 +66,7 @@ import java.io.File; import java.io.IOException; import java.io.PrintWriter; +import java.net.URI; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -74,6 +76,24 @@ public class IndexTaskTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( + new TimestampSpec( + "ts", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("ts", "dim", "val"), + false, + 0 + ); + private final IndexSpec indexSpec; private final ObjectMapper jsonMapper; private IndexMerger indexMerger; @@ -107,7 +127,7 @@ public void testDeterminePartitions() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, 2, null, null, false, false), + createIngestionSpec(tmpDir, null, null, 2, null, null, false, false, true), null, jsonMapper ); @@ -118,13 +138,15 @@ public void testDeterminePartitions() throws Exception Assert.assertEquals("test", segments.get(0).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(HashBasedNumberedShardSpec.class)); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); + Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getPartitions()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(HashBasedNumberedShardSpec.class)); Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); + Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getPartitions()); } @Test @@ -143,7 +165,7 @@ public void testForceExtendableShardSpecs() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, 2, null, null, true, false), + createIngestionSpec(tmpDir, null, null, 2, null, null, true, false, false), null, jsonMapper ); @@ -154,12 +176,12 @@ public void testForceExtendableShardSpecs() throws Exception Assert.assertEquals("test", segments.get(0).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(NumberedShardSpec.class)); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(NumberedShardSpec.class)); Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); } @@ -181,15 +203,17 @@ public void testWithArbitraryGranularity() throws Exception null, createIngestionSpec( tmpDir, + null, new ArbitraryGranularitySpec( Granularities.MINUTE, - Arrays.asList(new Interval("2014/2015")) + Collections.singletonList(new Interval("2014/2015")) ), 10, null, null, false, - false + false, + true ), null, jsonMapper @@ -217,16 +241,18 @@ public void testIntervalBucketing() throws Exception null, createIngestionSpec( tmpDir, + null, new UniformGranularitySpec( Granularities.HOUR, Granularities.HOUR, - Arrays.asList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z")) + Collections.singletonList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z")) ), 50, null, null, false, - false + false, + true ), null, jsonMapper @@ -252,7 +278,7 @@ public void testNumShardsProvided() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, null, 1, null, false, false), + createIngestionSpec(tmpDir, null, null, null, 1, null, false, false, true), null, jsonMapper ); @@ -283,7 +309,7 @@ public void testAppendToExisting() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, 2, null, null, false, true), + createIngestionSpec(tmpDir, null, null, 2, null, null, false, true, false), null, jsonMapper ); @@ -321,6 +347,7 @@ public void testIntervalNotSpecified() throws Exception null, createIngestionSpec( tmpDir, + null, new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -330,7 +357,8 @@ public void testIntervalNotSpecified() throws Exception null, null, false, - false + false, + true ), null, jsonMapper @@ -342,30 +370,146 @@ public void testIntervalNotSpecified() throws Exception Assert.assertEquals("test", segments.get(0).getDataSource()); Assert.assertEquals(new Interval("2014-01-01T00/PT1H"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(NoneShardSpec.class)); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(new Interval("2014-01-01T01/PT1H"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(NoneShardSpec.class)); Assert.assertEquals(0, segments.get(1).getShardSpec().getPartitionNum()); Assert.assertEquals("test", segments.get(2).getDataSource()); Assert.assertEquals(new Interval("2014-01-01T02/PT1H"), segments.get(2).getInterval()); - Assert.assertTrue(segments.get(2).getShardSpec().getClass().equals(LinearShardSpec.class)); + Assert.assertTrue(segments.get(2).getShardSpec().getClass().equals(NoneShardSpec.class)); Assert.assertEquals(0, segments.get(2).getShardSpec().getPartitionNum()); } @Test - public void testWithSmallPersistedSegmentBytes() throws Exception + public void testCSVFileWithHeader() throws Exception { File tmpDir = temporaryFolder.newFolder(); + File tmpFile = File.createTempFile("druid", "index", tmpDir); PrintWriter writer = new PrintWriter(tmpFile); + writer.println("time,d,val"); writer.println("2014-01-01T00:00:10Z,a,1"); - writer.println("2014-01-01T01:00:20Z,b,1"); - writer.println("2014-01-01T02:00:30Z,c,1"); + + writer.close(); + + IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + null, + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + null, + true, + 0 + ), + null, + 2, + null, + null, + false, + false, + true + ), + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + + Assert.assertEquals(1, segments.size()); + + Assert.assertEquals(Arrays.asList("d"), segments.get(0).getDimensions()); + Assert.assertEquals(Arrays.asList("val"), segments.get(0).getMetrics()); + Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); + } + + @Test + public void testCSVFileWithHeaderColumnOverride() throws Exception + { + File tmpDir = temporaryFolder.newFolder(); + + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + PrintWriter writer = new PrintWriter(tmpFile); + writer.println("time,d,val"); + writer.println("2014-01-01T00:00:10Z,a,1"); + + writer.close(); + + IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + null, + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("time", "dim", "val"), + true, + 0 + ), + null, + 2, + null, + null, + false, + false, + true + ), + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + + Assert.assertEquals(1, segments.size()); + + Assert.assertEquals(Arrays.asList("dim"), segments.get(0).getDimensions()); + Assert.assertEquals(Arrays.asList("val"), segments.get(0).getMetrics()); + Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); + } + + @Test + public void testWithSmallMaxTotalRows() throws Exception + { + File tmpDir = temporaryFolder.newFolder(); + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + PrintWriter writer = new PrintWriter(tmpFile); + writer.println("2014-01-01T00:00:10Z,a,1"); + writer.println("2014-01-01T00:00:10Z,b,2"); + writer.println("2014-01-01T00:00:10Z,c,3"); + writer.println("2014-01-01T01:00:20Z,a,1"); + writer.println("2014-01-01T01:00:20Z,b,2"); + writer.println("2014-01-01T01:00:20Z,c,d"); + writer.println("2014-01-01T02:00:30Z,a,1"); + writer.println("2014-01-01T02:00:30Z,b,2"); + writer.println("2014-01-01T02:00:30Z,c,3"); writer.close(); IndexTask indexTask = new IndexTask( @@ -373,6 +517,7 @@ public void testWithSmallPersistedSegmentBytes() throws Exception null, createIngestionSpec( tmpDir, + null, new UniformGranularitySpec( Granularities.HOUR, Granularities.MINUTE, @@ -380,7 +525,9 @@ public void testWithSmallPersistedSegmentBytes() throws Exception ), 2, null, - 10L, + 2, + 2, + false, false, false ), @@ -390,22 +537,18 @@ public void testWithSmallPersistedSegmentBytes() throws Exception final List segments = runTask(indexTask); - Assert.assertEquals(3, segments.size()); - - Assert.assertEquals("test", segments.get(0).getDataSource()); - Assert.assertEquals(new Interval("2014-01-01T00/PT1H"), segments.get(0).getInterval()); - Assert.assertTrue(segments.get(0).getShardSpec().getClass().equals(LinearShardSpec.class)); - Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); + Assert.assertEquals(6, segments.size()); - Assert.assertEquals("test", segments.get(1).getDataSource()); - Assert.assertEquals(new Interval("2014-01-01T01/PT1H"), segments.get(1).getInterval()); - Assert.assertTrue(segments.get(1).getShardSpec().getClass().equals(LinearShardSpec.class)); - Assert.assertEquals(0, segments.get(1).getShardSpec().getPartitionNum()); + for (int i = 0; i < 6; i++) { + final DataSegment segment = segments.get(i); + final Interval expectedInterval = new Interval(String.format("2014-01-01T0%d/PT1H", (i / 2))); + final int expectedPartitionNum = i % 2; - Assert.assertEquals("test", segments.get(2).getDataSource()); - Assert.assertEquals(new Interval("2014-01-01T02/PT1H"), segments.get(2).getInterval()); - Assert.assertTrue(segments.get(2).getShardSpec().getClass().equals(LinearShardSpec.class)); - Assert.assertEquals(0, segments.get(2).getShardSpec().getPartitionNum()); + Assert.assertEquals("test", segment.getDataSource()); + Assert.assertEquals(expectedInterval, segment.getInterval()); + Assert.assertTrue(segment.getShardSpec().getClass().equals(NumberedShardSpec.class)); + Assert.assertEquals(expectedPartitionNum, segment.getShardSpec().getPartitionNum()); + } } private final List runTask(final IndexTask indexTask) throws Exception @@ -420,7 +563,7 @@ null, null, new TaskActionClient() public RetType submit(TaskAction taskAction) throws IOException { if (taskAction instanceof LockListAction) { - return (RetType) Arrays.asList( + return (RetType) Collections.singletonList( new TaskLock( "", "", null, new DateTime().toString() ) @@ -473,7 +616,13 @@ public DataSegment push(File file, DataSegment segment) throws IOException segments.add(segment); return segment; } - }, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }, null, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), indexMerger, indexIO, null, null, indexMergerV9 ) ); @@ -485,12 +634,41 @@ public DataSegment push(File file, DataSegment segment) throws IOException private IndexTask.IndexIngestionSpec createIngestionSpec( File baseDir, + ParseSpec parseSpec, + GranularitySpec granularitySpec, + Integer targetPartitionSize, + Integer numShards, + Integer maxTotalRows, + boolean forceExtendableShardSpecs, + boolean appendToExisting, + boolean forceGuaranteedRollup + ) + { + return createIngestionSpec( + baseDir, + parseSpec, + granularitySpec, + targetPartitionSize, + numShards, + 1, + maxTotalRows, + forceExtendableShardSpecs, + appendToExisting, + forceGuaranteedRollup + ); + } + + private IndexTask.IndexIngestionSpec createIngestionSpec( + File baseDir, + ParseSpec parseSpec, GranularitySpec granularitySpec, Integer targetPartitionSize, Integer numShards, - Long persistedSegmentsBytes, + Integer maxRowsInMemory, + Integer maxTotalRows, boolean forceExtendableShardSpecs, - boolean appendToExisting + boolean appendToExisting, + boolean forceGuaranteedRollup ) { return new IndexTask.IndexIngestionSpec( @@ -498,20 +676,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( "test", jsonMapper.convertValue( new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), - Lists.newArrayList(), - Lists.newArrayList() - ), - null, - Arrays.asList("ts", "dim", "val") - ), + parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC, null ), Map.class @@ -531,19 +696,21 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( baseDir, "druid*", null - ), appendToExisting, null + ), appendToExisting ), new IndexTask.IndexTuningConfig( targetPartitionSize, - 1, - persistedSegmentsBytes, + maxRowsInMemory, + maxTotalRows, null, numShards, indexSpec, null, true, forceExtendableShardSpecs, - true + forceGuaranteedRollup, + null, + null ) ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 7f4464f7d540..dd517566f565 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -101,6 +101,7 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; @@ -219,7 +220,7 @@ public TestFirehoseFactory() } @Override - public Firehose connect(InputRowParser parser) throws IOException, ParseException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException, ParseException { return new TestFirehose(); } @@ -1042,6 +1043,7 @@ Map> getHandOffCallbacks() null, // DataSegmentMover null, // DataSegmentArchiver new TestDataSegmentAnnouncer(), + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, conglomerate, MoreExecutors.sameThreadExecutor(), // queryExecutorService diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index 09d1973c2245..9f2bd7f78232 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -54,8 +54,10 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; public class SameIntervalMergeTaskTest @@ -200,7 +202,13 @@ public DataSegment push(File file, DataSegment segment) throws IOException segments.add(segment); return segment; } - }, null, null, null, null, null, null, null, null, new SegmentLoader() + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + return null; + } + + }, null, null, null, null, null, null, null, null, null, new SegmentLoader() { @Override public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 7db77f87a734..80ca2faccbb9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -88,7 +88,6 @@ public void testIndexTaskIOConfigDefaults() throws Exception ); Assert.assertEquals(false, ioConfig.isAppendToExisting()); - Assert.assertEquals(false, ioConfig.isSkipFirehoseCaching()); } @Test @@ -186,8 +185,8 @@ public void testIndexTaskSerde() throws Exception ), jsonMapper ), - new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true, true), - new IndexTask.IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, true) + new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), + new IndexTask.IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, true, null, null) ), null, jsonMapper @@ -210,7 +209,6 @@ public void testIndexTaskSerde() throws Exception Assert.assertTrue(taskIoConfig.getFirehoseFactory() instanceof LocalFirehoseFactory); Assert.assertTrue(task2IoConfig.getFirehoseFactory() instanceof LocalFirehoseFactory); Assert.assertEquals(taskIoConfig.isAppendToExisting(), task2IoConfig.isAppendToExisting()); - Assert.assertEquals(taskIoConfig.isSkipFirehoseCaching(), task2IoConfig.isSkipFirehoseCaching()); IndexTask.IndexTuningConfig taskTuningConfig = task.getIngestionSchema().getTuningConfig(); IndexTask.IndexTuningConfig task2TuningConfig = task2.getIngestionSchema().getTuningConfig(); @@ -251,8 +249,8 @@ public void testIndexTaskwithResourceSerde() throws Exception ), jsonMapper ), - new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true) + new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true, null, null) ), null, jsonMapper diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index b7431f6f3abd..71b656e57b44 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -93,6 +93,7 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; @@ -231,6 +232,12 @@ public DataSegment push(File file, DataSegment segment) throws IOException { return segment; } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } }, new DataSegmentKiller() { @@ -270,6 +277,7 @@ public DataSegment restore(DataSegment segment) throws SegmentLoadingException } }, null, // segment announcer + null, notifierFactory, null, // query runner factory conglomerate corporation unionized collective null, // query executor service @@ -518,7 +526,7 @@ public void simpleFirehoseReadingTest() throws IOException Integer rowcount = 0; try (final IngestSegmentFirehose firehose = (IngestSegmentFirehose) - factory.connect(rowParser)) { + factory.connect(rowParser, null)) { while (firehose.hasMore()) { InputRow row = firehose.nextRow(); Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 165b29fc7fee..7b9b1c972540 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -144,7 +144,7 @@ public void testSimple() throws Exception int count = 0; long sum = 0; - try (final Firehose firehose = factory.connect(ROW_PARSER)) { + try (final Firehose firehose = factory.connect(ROW_PARSER, null)) { while (firehose.hasMore()) { final InputRow row = firehose.nextRow(); count++; @@ -318,6 +318,7 @@ public TaskActionClient create(Task task) null, // segment mover null, // segment archiver null, // segment announcer, + null, notifierFactory, null, // query runner factory conglomerate corporation unionized collective null, // query executor service diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java index 57ae6d155ada..fb870d279f83 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerRunPendingTasksConcurrencyTest.java @@ -65,6 +65,7 @@ public void testConcurrency() throws Exception remoteTaskRunner = rtrTestUtils.makeRemoteTaskRunner( new TestRemoteTaskRunnerConfig(new Period("PT3600S")) { + @Override public int getPendingTasksRunnerNumThreads() { return 2; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index fd2770fd2643..d8abc041c421 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -102,6 +102,7 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -121,6 +122,7 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; @@ -238,7 +240,7 @@ private static InputRow IR(String dt, String dim1, String dim2, float met) private static class MockExceptionalFirehoseFactory implements FirehoseFactory { @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { return new Firehose() { @@ -288,7 +290,7 @@ public MockFirehoseFactory(@JsonProperty("usedByRealtimeIdxTask") boolean usedBy } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { final Iterator inputRowIterator = usedByRealtimeIdxTask ? realtimeIdxTaskInputRows.iterator() @@ -476,6 +478,12 @@ public DataSegment push(File file, DataSegment segment) throws IOException pushedSegments++; return segment; } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } }; } @@ -566,13 +574,8 @@ public void unannounceSegments(Iterable segments) throws IOExceptio { } - - @Override - public boolean isAnnounced(DataSegment segment) - { - return false; - } }, // segment announcer + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective MoreExecutors.sameThreadExecutor(), // query executor service @@ -652,8 +655,8 @@ public void testIndexTask() throws Exception ), mapper ), - new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true) + new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true, null, null) ), null, MAPPER @@ -710,8 +713,8 @@ public void testIndexTaskFailure() throws Exception ), mapper ), - new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true) + new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false), + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true, null, null) ), null, MAPPER @@ -1021,6 +1024,12 @@ public DataSegment push(File file, DataSegment dataSegment) throws IOException { throw new RuntimeException("FAILURE"); } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } }; tb = setUpTaskToolboxFactory(dataSegmentPusher, handoffNotifierFactory, mdc); @@ -1069,8 +1078,8 @@ public void testResumeTasks() throws Exception ), mapper ), - new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false, null), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null) + new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null) ), null, MAPPER diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java index bd131e006f64..3681aa71e7d1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java @@ -128,7 +128,7 @@ public void testScale() Assert.assertEquals(created.getNodeIds().size(), 1); Assert.assertEquals("theInstance", created.getNodeIds().get(0)); - AutoScalingData deleted = autoScaler.terminate(Arrays.asList("dummyIP")); + AutoScalingData deleted = autoScaler.terminate(Collections.singletonList("dummyIP")); Assert.assertEquals(deleted.getNodeIds().size(), 1); Assert.assertEquals(INSTANCE_ID, deleted.getNodeIds().get(0)); @@ -185,7 +185,7 @@ public void testIptoIdLookup() throws Exception ); EasyMock.replay(describeInstancesResult); - EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).times(n); + EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).times(n); EasyMock.replay(reservation); List ids = autoScaler.ipToIdLookup(ips); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordRedirectInfoTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordRedirectInfoTest.java index 42cfb12de39b..a0db2b1bebdd 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordRedirectInfoTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordRedirectInfoTest.java @@ -40,18 +40,33 @@ public void setUp() } @Test - public void testDoLocal() + public void testDoLocalWhenLeading() { - EasyMock.expect(taskMaster.isLeading()).andReturn(true).anyTimes(); + EasyMock.expect(taskMaster.isLeader()).andReturn(true).anyTimes(); EasyMock.replay(taskMaster); Assert.assertTrue(redirectInfo.doLocal(null)); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/leader")); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/isLeader")); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/other/path")); + EasyMock.verify(taskMaster); + } + + @Test + public void testDoLocalWhenNotLeading() + { + EasyMock.expect(taskMaster.isLeader()).andReturn(false).anyTimes(); + EasyMock.replay(taskMaster); + Assert.assertFalse(redirectInfo.doLocal(null)); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/leader")); + Assert.assertTrue(redirectInfo.doLocal("/druid/indexer/v1/isLeader")); + Assert.assertFalse(redirectInfo.doLocal("/druid/indexer/v1/other/path")); EasyMock.verify(taskMaster); } @Test public void testGetRedirectURLNull() { - EasyMock.expect(taskMaster.getLeader()).andReturn(null).anyTimes(); + EasyMock.expect(taskMaster.getCurrentLeader()).andReturn(null).anyTimes(); EasyMock.replay(taskMaster); URL url = redirectInfo.getRedirectURL("query", "/request"); Assert.assertNull(url); @@ -61,7 +76,7 @@ public void testGetRedirectURLNull() @Test public void testGetRedirectURLEmpty() { - EasyMock.expect(taskMaster.getLeader()).andReturn("").anyTimes(); + EasyMock.expect(taskMaster.getCurrentLeader()).andReturn("").anyTimes(); EasyMock.replay(taskMaster); URL url = redirectInfo.getRedirectURL("query", "/request"); Assert.assertNull(url); @@ -74,7 +89,7 @@ public void testGetRedirectURL() String host = "localhost"; String query = "foo=bar&x=y"; String request = "/request"; - EasyMock.expect(taskMaster.getLeader()).andReturn(host).anyTimes(); + EasyMock.expect(taskMaster.getCurrentLeader()).andReturn(host).anyTimes(); EasyMock.replay(taskMaster); URL url = redirectInfo.getRedirectURL(query, request); Assert.assertEquals("http://localhost/request?foo=bar&x=y", url.toString()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 173bd905c37a..08b341668dd0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import io.druid.indexing.common.TaskLocation; @@ -79,7 +80,10 @@ public void setUp() throws Exception null, new AuthConfig(true) ); + } + public void expectAuthorizationTokenCheck() + { EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn( new AuthorizationInfo() { @@ -98,9 +102,40 @@ public Access isAuthorized( ); } + @Test + public void testLeader() + { + EasyMock.expect(taskMaster.getCurrentLeader()).andReturn("boz").once(); + EasyMock.replay(taskRunner, taskMaster, tsqa, req); + + final Response response = overlordResource.getLeader(); + Assert.assertEquals("boz", response.getEntity()); + Assert.assertEquals(200, response.getStatus()); + } + + @Test + public void testIsLeader() + { + EasyMock.expect(taskMaster.isLeader()).andReturn(true).once(); + EasyMock.expect(taskMaster.isLeader()).andReturn(false).once(); + EasyMock.replay(taskRunner, taskMaster, tsqa, req); + + // true + final Response response1 = overlordResource.isLeader(); + Assert.assertEquals(ImmutableMap.of("leader", true), response1.getEntity()); + Assert.assertEquals(200, response1.getStatus()); + + // false + final Response response2 = overlordResource.isLeader(); + Assert.assertEquals(ImmutableMap.of("leader", false), response2.getEntity()); + Assert.assertEquals(404, response2.getStatus()); + } + @Test public void testSecuredGetWaitingTask() throws Exception { + expectAuthorizationTokenCheck(); + EasyMock.expect(tsqa.getActiveTasks()).andReturn( ImmutableList.of( getTaskWithIdAndDatasource("id_1", "allow"), @@ -128,6 +163,8 @@ public void testSecuredGetWaitingTask() throws Exception @Test public void testSecuredGetCompleteTasks() { + expectAuthorizationTokenCheck(); + List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); EasyMock.expect(tsqa.getRecentlyFinishedTaskStatuses()).andReturn( Lists.transform( @@ -165,6 +202,8 @@ public TaskStatus apply(String input) @Test public void testSecuredGetRunningTasks() { + expectAuthorizationTokenCheck(); + List tasksIds = ImmutableList.of("id_1", "id_2"); EasyMock.>expect(taskRunner.getRunningTasks()).andReturn( ImmutableList.of( @@ -191,6 +230,8 @@ public void testSecuredGetRunningTasks() @Test public void testSecuredTaskPost() { + expectAuthorizationTokenCheck(); + EasyMock.replay(taskRunner, taskMaster, tsqa, req); Task task = NoopTask.create(); Response response = overlordResource.taskPost(task, req); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index aef09f121cad..61e0353b4bc1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -198,11 +198,11 @@ public void testOverlordRun() throws Exception // basic task master lifecycle test taskMaster.start(); announcementLatch.await(); - while (!taskMaster.isLeading()) { + while (!taskMaster.isLeader()) { // I believe the control will never reach here and thread will never sleep but just to be on safe side Thread.sleep(10); } - Assert.assertEquals(taskMaster.getLeader(), druidNode.getHostAndPort()); + Assert.assertEquals(taskMaster.getCurrentLeader(), druidNode.getHostAndPort()); // Test Overlord resource stuff overlordResource = new OverlordResource( taskMaster, @@ -271,7 +271,7 @@ public void testOverlordRun() throws Exception response = overlordResource.getCompleteTasks(req); Assert.assertEquals(2, (((List) response.getEntity()).size())); taskMaster.stop(); - Assert.assertFalse(taskMaster.isLeading()); + Assert.assertFalse(taskMaster.isLeader()); EasyMock.verify(taskLockbox, taskActionClientFactory); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java index e53dc25d27a3..aad319300edf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java @@ -69,6 +69,42 @@ public String getDataSource() Assert.assertEquals("lhost", worker.getWorker().getHost()); } + @Test + public void testFindWorkerForTaskWhenSameCurrCapacityUsed() throws Exception + { + final EqualDistributionWorkerSelectStrategy strategy = new EqualDistributionWorkerSelectStrategy(); + + Optional optional = strategy.findWorkerForTask( + new RemoteTaskRunnerConfig(), + ImmutableMap.of( + "lhost", + new ImmutableWorkerInfo( + new Worker("lhost", "lhost", 5, "v1"), 5, + Sets.newHashSet(), + Sets.newHashSet(), + DateTime.now() + ), + "localhost", + new ImmutableWorkerInfo( + new Worker("localhost", "localhost", 10, "v1"), 5, + Sets.newHashSet(), + Sets.newHashSet(), + DateTime.now() + ) + ), + new NoopTask(null, 1, 0, null, null, null) + { + @Override + public String getDataSource() + { + return "foo"; + } + } + ); + ImmutableWorkerInfo worker = optional.get(); + Assert.assertEquals("localhost", worker.getWorker().getHost()); + } + @Test public void testOneDisableWorkerDifferentUsedCapacity() throws Exception { @@ -78,28 +114,28 @@ public void testOneDisableWorkerDifferentUsedCapacity() throws Exception Optional optional = strategy.findWorkerForTask( new RemoteTaskRunnerConfig(), ImmutableMap.of( - "lhost", - new ImmutableWorkerInfo( - new Worker("disableHost", "disableHost", 10, DISABLED_VERSION), 2, - Sets.newHashSet(), - Sets.newHashSet(), - DateTime.now() - ), - "localhost", - new ImmutableWorkerInfo( - new Worker("enableHost", "enableHost", 10, "v1"), 5, - Sets.newHashSet(), - Sets.newHashSet(), - DateTime.now() - ) + "lhost", + new ImmutableWorkerInfo( + new Worker("disableHost", "disableHost", 10, DISABLED_VERSION), 2, + Sets.newHashSet(), + Sets.newHashSet(), + DateTime.now() + ), + "localhost", + new ImmutableWorkerInfo( + new Worker("enableHost", "enableHost", 10, "v1"), 5, + Sets.newHashSet(), + Sets.newHashSet(), + DateTime.now() + ) ), new NoopTask(null, 1, 0, null, null, null) { @Override public String getDataSource() - { - return "foo"; - } + { + return "foo"; + } } ); ImmutableWorkerInfo worker = optional.get(); @@ -113,31 +149,31 @@ public void testOneDisableWorkerSameUsedCapacity() throws Exception final EqualDistributionWorkerSelectStrategy strategy = new EqualDistributionWorkerSelectStrategy(); Optional optional = strategy.findWorkerForTask( - new RemoteTaskRunnerConfig(), - ImmutableMap.of( - "lhost", - new ImmutableWorkerInfo( - new Worker("disableHost", "disableHost", 10, DISABLED_VERSION), 5, - Sets.newHashSet(), - Sets.newHashSet(), - DateTime.now() - ), - "localhost", - new ImmutableWorkerInfo( - new Worker("enableHost", "enableHost", 10, "v1"), 5, - Sets.newHashSet(), - Sets.newHashSet(), - DateTime.now() - ) + new RemoteTaskRunnerConfig(), + ImmutableMap.of( + "lhost", + new ImmutableWorkerInfo( + new Worker("disableHost", "disableHost", 10, DISABLED_VERSION), 5, + Sets.newHashSet(), + Sets.newHashSet(), + DateTime.now() ), - new NoopTask(null, 1, 0, null, null, null) - { - @Override - public String getDataSource() - { - return "foo"; - } - } + "localhost", + new ImmutableWorkerInfo( + new Worker("enableHost", "enableHost", 10, "v1"), 5, + Sets.newHashSet(), + Sets.newHashSet(), + DateTime.now() + ) + ), + new NoopTask(null, 1, 0, null, null, null) + { + @Override + public String getDataSource() + { + return "foo"; + } + } ); ImmutableWorkerInfo worker = optional.get(); Assert.assertEquals("enableHost", worker.getWorker().getHost()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index 41ea2329263e..6778deb7130d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -269,7 +269,7 @@ public void testResetSupervisor() throws Exception verifyAll(); } - private class TestSupervisorSpec implements SupervisorSpec + private static class TestSupervisorSpec implements SupervisorSpec { private final String id; private final Supervisor supervisor; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index ad09472d6af6..c593f6986c05 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -307,7 +307,7 @@ public void testReset() throws Exception verifyAll(); } - private class TestSupervisorSpec implements SupervisorSpec + private static class TestSupervisorSpec implements SupervisorSpec { private final String id; private final Supervisor supervisor; diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java index 2e36d2a18184..be5ef6df9ea2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java @@ -59,12 +59,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } } - @Override - public boolean isAnnounced(DataSegment segment) - { - return announcedSegments.contains(segment); - } - public Set getAnnouncedSegments() { return ImmutableSet.copyOf(announcedSegments); diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java index ad99827052d2..923a80299605 100644 --- a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java @@ -26,6 +26,8 @@ import java.io.File; import java.io.IOException; +import java.net.URI; +import java.util.Map; import java.util.Set; public class TestDataSegmentPusher implements DataSegmentPusher @@ -52,6 +54,12 @@ public DataSegment push(File file, DataSegment segment) throws IOException return segment; } + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + public Set getPushedSegments() { return ImmutableSet.copyOf(pushedSegments); diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java b/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java index e79201cc62bb..3a5af4805634 100644 --- a/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java @@ -131,4 +131,16 @@ public Iterable getInventory() { return null; } + + @Override + public boolean isStarted() + { + return true; + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + return false; + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 3ac1cad36e32..05e48011b84b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -169,7 +169,7 @@ private WorkerTaskMonitor createTaskMonitor() new TaskToolboxFactory( taskConfig, taskActionClientFactory, - null, null, null, null, null, null, notifierFactory, null, null, null, new SegmentLoaderFactory( + null, null, null, null, null, null, null, notifierFactory, null, null, null, new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager( null, new SegmentLoaderConfig() diff --git a/integration-tests/docker/Dockerfile b/integration-tests/docker/Dockerfile index 2b81c5ad39b4..1d98753b00e0 100644 --- a/integration-tests/docker/Dockerfile +++ b/integration-tests/docker/Dockerfile @@ -23,8 +23,9 @@ RUN wget -q -O - http://www.us.apache.org/dist/zookeeper/zookeeper-3.4.6/zookeep && ln -s /usr/local/zookeeper-3.4.6 /usr/local/zookeeper # Kafka -RUN wget -q -O - http://www.us.apache.org/dist/kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz | tar -xzf - -C /usr/local \ - && ln -s /usr/local/kafka_2.10-0.9.0.1 /usr/local/kafka +# Match the version to the Kafka client used by KafkaSupervisor +RUN wget -q -O - http://www.us.apache.org/dist/kafka/0.10.2.0/kafka_2.11-0.10.2.0.tgz | tar -xzf - -C /usr/local \ + && ln -s /usr/local/kafka_2.11-0.10.2.0 /usr/local/kafka # Druid system user RUN adduser --system --group --no-create-home druid \ @@ -62,7 +63,7 @@ ADD supervisord.conf /etc/supervisor/conf.d/supervisord.conf # unless advertised.host.name is set to docker host ip, publishing data fails # run this last to avoid rebuilding the image every time the ip changes ADD docker_ip docker_ip -RUN perl -pi -e "s/#advertised.port=.*/advertised.port=9092/; s/#advertised.host.*/advertised.host.name=$(cat docker_ip)/" /usr/local/kafka/config/server.properties +RUN perl -pi -e "s/#advertised.listeners=.*/advertised.listeners=PLAINTEXT:\/\/$(cat docker_ip):9092/" /usr/local/kafka/config/server.properties # Expose ports: # - 8081: HTTP (coordinator) diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index d64b5d35271d..981811065f9f 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -30,8 +30,7 @@ - 0.8.2.1 - 0.4 + 0.10.2.0 @@ -54,6 +53,12 @@ io.druid.extensions druid-kafka-eight ${project.parent.version} + + + kafka_2.10 + org.apache.kafka + + io.druid.extensions @@ -69,6 +74,12 @@ io.druid.extensions druid-kafka-indexing-service ${project.parent.version} + + + org.apache.kafka + kafka-clients + + io.druid @@ -91,34 +102,21 @@ easymock test - - com.101tec - zkclient - ${zkclient.version} - - - org.apache.zookeeper - zookeeper - - - org.apache.kafka - kafka_2.10 + kafka_2.11 ${apache.kafka.version} - - org.slf4j + log4j + log4j + + slf4j-log4j12 + org.slf4j - - org.apache.kafka - kafka-clients - 0.9.0.1 - diff --git a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java index ee7cda2e6840..05836d66b714 100644 --- a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java +++ b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java @@ -33,7 +33,6 @@ import com.metamx.http.client.auth.BasicCredentials; import io.druid.curator.CuratorConfig; import io.druid.guice.JsonConfigProvider; -import io.druid.guice.LazySingleton; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Client; import io.druid.testing.IntegrationTestingConfig; @@ -72,7 +71,7 @@ public HttpClient getHttpClient( } @Provides - @LazySingleton + @ManageLifecycle public ServiceEmitter getServiceEmitter(Supplier config, ObjectMapper jsonMapper) { return new ServiceEmitter("", "", new LoggingEmitter(config.get(), jsonMapper)); diff --git a/integration-tests/src/main/java/io/druid/testing/utils/TestQueryHelper.java b/integration-tests/src/main/java/io/druid/testing/utils/TestQueryHelper.java index f0369d31bc5c..4ce8cac031ea 100644 --- a/integration-tests/src/main/java/io/druid/testing/utils/TestQueryHelper.java +++ b/integration-tests/src/main/java/io/druid/testing/utils/TestQueryHelper.java @@ -21,10 +21,15 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.inject.Inject; - import io.druid.java.util.common.ISE; +import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.logger.Logger; +import io.druid.query.Druids; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; import io.druid.testing.IntegrationTestingConfig; import io.druid.testing.clients.QueryResourceTestClient; @@ -118,4 +123,28 @@ private String getBrokerURL() { return String.format("%s/druid/v2?pretty", broker); } + + @SuppressWarnings("unchecked") + public int countRows(String dataSource, String interval) + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("rows", "count") + ) + ) + .granularity(Granularities.ALL) + .intervals(interval) + .build(); + + List> results = queryClient.query(getBrokerURL(), query); + if (results.isEmpty()) { + return 0; + } else { + Map map = (Map) results.get(0).get("result"); + + return (Integer) map.get("rows"); + } + } } diff --git a/integration-tests/src/main/java/org/testng/TestNG.java b/integration-tests/src/main/java/org/testng/TestNG.java index 6e7fc4ac7982..f4076fc34037 100644 --- a/integration-tests/src/main/java/org/testng/TestNG.java +++ b/integration-tests/src/main/java/org/testng/TestNG.java @@ -954,10 +954,10 @@ private void initializeCommandLineSuitesGroups() } } - private void addReporter(Class r) + private void addReporter(Class reporterClass) { - if (!m_reporters.contains(r)) { - m_reporters.add(ClassHelper.newInstance(r)); + if (m_reporters.stream().noneMatch(reporterClass::isInstance)) { + m_reporters.add(ClassHelper.newInstance(reporterClass)); } } @@ -2211,7 +2211,7 @@ public void setGroupByInstances(boolean b) private URLClassLoader m_serviceLoaderClassLoader; private List m_serviceLoaderListeners = Lists.newArrayList(); - /* + /** * Used to test ServiceClassLoader */ public void setServiceLoaderClassLoader(URLClassLoader ucl) @@ -2219,7 +2219,7 @@ public void setServiceLoaderClassLoader(URLClassLoader ucl) m_serviceLoaderClassLoader = ucl; } - /* + /** * Used to test ServiceClassLoader */ private void addServiceLoaderListener(ITestNGListener l) @@ -2227,7 +2227,7 @@ private void addServiceLoaderListener(ITestNGListener l) m_serviceLoaderListeners.add(l); } - /* + /** * Used to test ServiceClassLoader */ public List getServiceLoaderListeners() diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java index ef5328fc3db1..defa334251c5 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java @@ -21,7 +21,6 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; - import io.druid.java.util.common.logger.Logger; import io.druid.testing.IntegrationTestingConfig; import io.druid.testing.clients.ClientInfoResourceTestClient; @@ -66,7 +65,7 @@ public void testIndexData() throws Exception Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot")); } catch (Exception e) { - e.printStackTrace(); + LOG.error(e, "Error while testing"); throw Throwables.propagate(e); } finally { diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java index e3580da9cff2..cdf7f5b946dc 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java @@ -28,9 +28,11 @@ import io.druid.testing.utils.RetryUtil; import io.druid.testing.utils.TestQueryHelper; import kafka.admin.AdminUtils; -import kafka.common.TopicExistsException; +import kafka.admin.RackAwareMode; import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkConnection; import org.apache.commons.io.IOUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -84,7 +86,8 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest private String supervisorId; private ZkClient zkClient; - private Boolean segmentsExist; // to tell if we should remove segments during teardown + private ZkUtils zkUtils; + private boolean segmentsExist; // to tell if we should remove segments during teardown // format for the querying interval private final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'"); @@ -112,13 +115,11 @@ public void testKafka() zkHosts, sessionTimeoutMs, connectionTimeoutMs, ZKStringSerializer$.MODULE$ ); + zkUtils = new ZkUtils(zkClient, new ZkConnection(zkHosts, sessionTimeoutMs), false); int numPartitions = 4; int replicationFactor = 1; Properties topicConfig = new Properties(); - AdminUtils.createTopic(zkClient, TOPIC_NAME, numPartitions, replicationFactor, topicConfig); - } - catch (TopicExistsException e) { - // it's ok if the topic already exists + AdminUtils.createTopic(zkUtils, TOPIC_NAME, numPartitions, replicationFactor, topicConfig, RackAwareMode.Disabled$.MODULE$); } catch (Exception e) { throw new ISE(e, "could not create kafka topic"); @@ -285,7 +286,7 @@ public void afterClass() throws Exception LOG.info("teardown"); // delete kafka topic - AdminUtils.deleteTopic(zkClient, TOPIC_NAME); + AdminUtils.deleteTopic(zkUtils, TOPIC_NAME); // remove segments if (segmentsExist) { diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java index dcebb3abfc7d..07501edf1766 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaTest.java @@ -28,9 +28,11 @@ import io.druid.testing.utils.RetryUtil; import io.druid.testing.utils.TestQueryHelper; import kafka.admin.AdminUtils; -import kafka.common.TopicExistsException; +import kafka.admin.RackAwareMode; import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkConnection; import org.apache.commons.io.IOUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -84,7 +86,8 @@ public class ITKafkaTest extends AbstractIndexerTest private String taskID; private ZkClient zkClient; - private Boolean segmentsExist; // to tell if we should remove segments during teardown + private ZkUtils zkUtils; + private boolean segmentsExist; // to tell if we should remove segments during teardown // format for the querying interval private final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'"); @@ -112,13 +115,11 @@ public void testKafka() zkHosts, sessionTimeoutMs, connectionTimeoutMs, ZKStringSerializer$.MODULE$ ); + zkUtils = new ZkUtils(zkClient, new ZkConnection(zkHosts, sessionTimeoutMs), false); int numPartitions = 1; int replicationFactor = 1; Properties topicConfig = new Properties(); - AdminUtils.createTopic(zkClient, TOPIC_NAME, numPartitions, replicationFactor, topicConfig); - } - catch (TopicExistsException e) { - // it's ok if the topic already exists + AdminUtils.createTopic(zkUtils, TOPIC_NAME, numPartitions, replicationFactor, topicConfig, RackAwareMode.Disabled$.MODULE$); } catch (Exception e) { throw new ISE(e, "could not create kafka topic"); @@ -196,7 +197,7 @@ public void testKafka() LOG.info("-------------SUBMITTED TASK"); // wait for the task to finish - indexer.waitUntilTaskCompletes (taskID, 20000, 30); + indexer.waitUntilTaskCompletes(taskID, 20000, 30); // wait for segments to be handed off try { @@ -263,7 +264,7 @@ public void afterClass() throws Exception LOG.info("teardown"); // delete kafka topic - AdminUtils.deleteTopic(zkClient, TOPIC_NAME); + AdminUtils.deleteTopic(zkUtils, TOPIC_NAME); // remove segments if (segmentsExist) { diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java index e3ca8726df63..dbdc49f01074 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java @@ -86,8 +86,21 @@ public void testUnionQuery() throws Exception postEvents(i); } - // sleep for a while to let the events ingested - TimeUnit.SECONDS.sleep(5); + // wait until all events are ingested + RetryUtil.retryUntil( + () -> { + for (int i = 0; i < numTasks; i++) { + if (queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01") < 5) { + return false; + } + } + return true; + }, + true, + 1000, + 100, + "Waiting all events are ingested" + ); // should hit the queries on realtime task LOG.info("Running Union Queries.."); @@ -110,7 +123,7 @@ public Boolean call() throws Exception } }, true, - 60000, + 10000, 10, "Real-time generated segments loaded" ); @@ -120,7 +133,7 @@ public Boolean call() throws Exception } catch (Exception e) { - e.printStackTrace(); + LOG.error(e, "Error while testing"); throw Throwables.propagate(e); } finally { diff --git a/java-util/src/main/java/io/druid/java/util/common/collect/CountingMap.java b/java-util/src/main/java/io/druid/java/util/common/collect/CountingMap.java deleted file mode 100644 index 6e7911c0b7fd..000000000000 --- a/java-util/src/main/java/io/druid/java/util/common/collect/CountingMap.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.java.util.common.collect; - -import com.google.common.base.Function; -import com.google.common.collect.Maps; - -import java.util.AbstractMap; -import java.util.HashMap; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; - -// Can't find a good way to abstract over which counter representation is used, -// so I just pick Long/MutableLong. -public class CountingMap extends AbstractMap -{ - private final HashMap counts = new HashMap<>(); - - public void add(K k, Long n) - { - if (!counts.containsKey(k)) { - counts.put(k, new AtomicLong(0)); - } - counts.get(k).addAndGet(n); - } - - public Set> entrySet() - { - return Maps.transformValues( - counts, - new Function() - { - @Override - public Long apply(AtomicLong n) - { - return n.get(); - } - } - ).entrySet(); - } -} diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/ScheduledExecutors.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/ScheduledExecutors.java index e8588f66c289..d636d320bdc7 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/ScheduledExecutors.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/ScheduledExecutors.java @@ -192,13 +192,7 @@ public static enum Signal public static ScheduledExecutorFactory createFactory(final Lifecycle lifecycle) { - return new ScheduledExecutorFactory() - { - public ScheduledExecutorService create(int corePoolSize, String nameFormat) - { - return ExecutorServices.manageLifecycle(lifecycle, fixed(corePoolSize, nameFormat)); - } - }; + return (corePoolSize, nameFormat) -> ExecutorServices.manageLifecycle(lifecycle, fixed(corePoolSize, nameFormat)); } public static ScheduledExecutorService fixed(int corePoolSize, String nameFormat) diff --git a/java-util/src/main/java/io/druid/java/util/common/granularity/GranularityType.java b/java-util/src/main/java/io/druid/java/util/common/granularity/GranularityType.java index e39bc404405a..89dfba515f5f 100644 --- a/java-util/src/main/java/io/druid/java/util/common/granularity/GranularityType.java +++ b/java-util/src/main/java/io/druid/java/util/common/granularity/GranularityType.java @@ -201,7 +201,7 @@ public static GranularityType fromPeriod(Period period) case 0: return GranularityType.YEAR; case 1: - if (vals[index] == 4) { + if (vals[index] == 3) { return GranularityType.QUARTER; } else if (vals[index] == 1) { diff --git a/java-util/src/main/java/io/druid/java/util/common/granularity/PeriodGranularity.java b/java-util/src/main/java/io/druid/java/util/common/granularity/PeriodGranularity.java index 2e86a46969cb..4708e855b6ae 100644 --- a/java-util/src/main/java/io/druid/java/util/common/granularity/PeriodGranularity.java +++ b/java-util/src/main/java/io/druid/java/util/common/granularity/PeriodGranularity.java @@ -109,19 +109,19 @@ public DateTimeFormatter getFormatter(Formatter type) @Override public DateTime increment(DateTime time) { - return new DateTime(increment(time.getMillis())); + return new DateTime(increment(time.getMillis()), getTimeZone()); } @Override public DateTime decrement(DateTime time) { - return new DateTime(decrement(time.getMillis())); + return new DateTime(decrement(time.getMillis()), getTimeZone()); } @Override public DateTime bucketStart(DateTime time) { - return new DateTime(truncate(time.getMillis())); + return new DateTime(truncate(time.getMillis()), getTimeZone()); } // Used only for Segments. Not for Queries diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterable.java b/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterable.java index fa83642e7c85..86e51f84dcab 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterable.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterable.java @@ -37,6 +37,7 @@ public DroppingIterable( this.numToDrop = numToDrop; } + @Override public Iterator iterator() { return new DroppingIterator<>(delegate.iterator(), numToDrop); diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterator.java b/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterator.java index da7e22db66ce..e65b72fe584d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterator.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/DroppingIterator.java @@ -38,6 +38,7 @@ public DroppingIterator( this.numToDrop = numToDrop; } + @Override public boolean hasNext() { if (! dropped) { @@ -50,6 +51,7 @@ public boolean hasNext() return delegate.hasNext(); } + @Override public T next() { if (! dropped) { @@ -61,6 +63,7 @@ public T next() return delegate.next(); } + @Override public void remove() { throw new UnsupportedOperationException(); diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterable.java b/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterable.java index d05f578399a6..f247fc594144 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterable.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterable.java @@ -58,6 +58,7 @@ public FunctionalIterable( this.delegate = delegate; } + @Override public Iterator iterator() { return delegate.iterator(); diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterator.java b/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterator.java index 2c9f21a21eb3..a829ae916dac 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterator.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/FunctionalIterator.java @@ -58,16 +58,19 @@ public FunctionalIterator( this.delegate = delegate; } + @Override public boolean hasNext() { return delegate.hasNext(); } + @Override public T next() { return delegate.next(); } + @Override public void remove() { delegate.remove(); diff --git a/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java b/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java index db19d25bd1e3..27f34f78370b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java +++ b/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java @@ -46,7 +46,7 @@ import java.nio.channels.Channels; import java.nio.channels.GatheringByteChannel; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -134,7 +134,7 @@ public void add(String name, File fileToAdd) throws IOException public void add(String name, ByteBuffer bufferToAdd) throws IOException { - add(name, Arrays.asList(bufferToAdd)); + add(name, Collections.singletonList(bufferToAdd)); } public void add(String name, List bufferToAdd) throws IOException diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java index e45ed7e6acc7..957e72560d2a 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java @@ -19,6 +19,7 @@ package io.druid.java.util.common.parsers; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Splitter; @@ -33,24 +34,17 @@ public class CSVParser implements Parser { - private final String listDelimiter; - private final Splitter listSplitter; - private final Function valueFunction; - - private final au.com.bytecode.opencsv.CSVParser parser = new au.com.bytecode.opencsv.CSVParser(); - - private ArrayList fieldNames = null; - - public CSVParser(final Optional listDelimiter) + private static final Function getValueFunction( + final String listDelimiter, + final Splitter listSplitter + ) { - this.listDelimiter = listDelimiter.isPresent() ? listDelimiter.get() : Parsers.DEFAULT_LIST_DELIMITER; - this.listSplitter = Splitter.on(this.listDelimiter); - this.valueFunction = new Function() + return new Function() { @Override public Object apply(String input) { - if (input.contains(CSVParser.this.listDelimiter)) { + if (input.contains(listDelimiter)) { return Lists.newArrayList( Iterables.transform( listSplitter.split(input), @@ -64,16 +58,49 @@ public Object apply(String input) }; } - public CSVParser(final Optional listDelimiter, final Iterable fieldNames) + private final String listDelimiter; + private final Splitter listSplitter; + private final Function valueFunction; + private final boolean hasHeaderRow; + private final int maxSkipHeaderRows; + + private final au.com.bytecode.opencsv.CSVParser parser = new au.com.bytecode.opencsv.CSVParser(); + + private ArrayList fieldNames = null; + private boolean hasParsedHeader = false; + private int skippedHeaderRows; + private boolean supportSkipHeaderRows; + + public CSVParser( + final Optional listDelimiter, + final boolean hasHeaderRow, + final int maxSkipHeaderRows + ) { - this(listDelimiter); + this.listDelimiter = listDelimiter.isPresent() ? listDelimiter.get() : Parsers.DEFAULT_LIST_DELIMITER; + this.listSplitter = Splitter.on(this.listDelimiter); + this.valueFunction = getValueFunction(this.listDelimiter, this.listSplitter); + + this.hasHeaderRow = hasHeaderRow; + this.maxSkipHeaderRows = maxSkipHeaderRows; + } + + public CSVParser( + final Optional listDelimiter, + final Iterable fieldNames, + final boolean hasHeaderRow, + final int maxSkipHeaderRows + ) + { + this(listDelimiter, hasHeaderRow, maxSkipHeaderRows); setFieldNames(fieldNames); } - public CSVParser(final Optional listDelimiter, final String header) + @VisibleForTesting + CSVParser(final Optional listDelimiter, final String header) { - this(listDelimiter); + this(listDelimiter, false, 0); setFieldNames(header); } @@ -83,6 +110,14 @@ public String getListDelimiter() return listDelimiter; } + @Override + public void startFileFromBeginning() + { + supportSkipHeaderRows = true; + hasParsedHeader = false; + skippedHeaderRows = 0; + } + @Override public List getFieldNames() { @@ -92,8 +127,10 @@ public List getFieldNames() @Override public void setFieldNames(final Iterable fieldNames) { - ParserUtils.validateFields(fieldNames); - this.fieldNames = Lists.newArrayList(fieldNames); + if (fieldNames != null) { + ParserUtils.validateFields(fieldNames); + this.fieldNames = Lists.newArrayList(fieldNames); + } } public void setFieldNames(final String header) @@ -109,9 +146,27 @@ public void setFieldNames(final String header) @Override public Map parse(final String input) { + if (!supportSkipHeaderRows && (hasHeaderRow || maxSkipHeaderRows > 0)) { + throw new UnsupportedOperationException("hasHeaderRow or maxSkipHeaderRows is not supported. " + + "Please check the indexTask supports these options."); + } + try { String[] values = parser.parseLine(input); + if (skippedHeaderRows < maxSkipHeaderRows) { + skippedHeaderRows++; + return null; + } + + if (hasHeaderRow && !hasParsedHeader) { + if (fieldNames == null) { + setFieldNames(Arrays.asList(values)); + } + hasParsedHeader = true; + return null; + } + if (fieldNames == null) { setFieldNames(ParserUtils.generateFieldNames(values.length)); } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java index c37c4bda989c..27114df31e95 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java @@ -19,6 +19,7 @@ package io.druid.java.util.common.parsers; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -36,16 +37,44 @@ public class DelimitedParser implements Parser { private static final String DEFAULT_DELIMITER = "\t"; + private static Function getValueFunction( + final String listDelimiter, + final Splitter listSplitter + ) + { + return (input) -> { + if (input.contains(listDelimiter)) { + return Lists.newArrayList( + Iterables.transform( + listSplitter.split(input), + ParserUtils.nullEmptyStringFunction + ) + ); + } else { + return ParserUtils.nullEmptyStringFunction.apply(input); + } + }; + } + private final String delimiter; private final String listDelimiter; - private final Splitter splitter; private final Splitter listSplitter; private final Function valueFunction; + private final boolean hasHeaderRow; + private final int maxSkipHeaderRows; private ArrayList fieldNames = null; + private boolean hasParsedHeader = false; + private int skippedHeaderRows; + private boolean supportSkipHeaderRows; - public DelimitedParser(final Optional delimiter, Optional listDelimiter) + public DelimitedParser( + final Optional delimiter, + final Optional listDelimiter, + final boolean hasHeaderRow, + final int maxSkipHeaderRows + ) { this.delimiter = delimiter.isPresent() ? delimiter.get() : DEFAULT_DELIMITER; this.listDelimiter = listDelimiter.isPresent() ? listDelimiter.get() : Parsers.DEFAULT_LIST_DELIMITER; @@ -58,40 +87,28 @@ public DelimitedParser(final Optional delimiter, Optional listDe this.splitter = Splitter.on(this.delimiter); this.listSplitter = Splitter.on(this.listDelimiter); - this.valueFunction = new Function() - { - @Override - public Object apply(String input) - { - if (input.contains(DelimitedParser.this.listDelimiter)) { - return Lists.newArrayList( - Iterables.transform( - listSplitter.split(input), - ParserUtils.nullEmptyStringFunction - ) - ); - } else { - return ParserUtils.nullEmptyStringFunction.apply(input); - } - } - }; + this.valueFunction = getValueFunction(this.listDelimiter, this.listSplitter); + this.hasHeaderRow = hasHeaderRow; + this.maxSkipHeaderRows = maxSkipHeaderRows; } public DelimitedParser( final Optional delimiter, final Optional listDelimiter, - final Iterable fieldNames + final Iterable fieldNames, + final boolean hasHeaderRow, + final int maxSkipHeaderRows ) { - this(delimiter, listDelimiter); + this(delimiter, listDelimiter, hasHeaderRow, maxSkipHeaderRows); setFieldNames(fieldNames); } - public DelimitedParser(final Optional delimiter, final Optional listDelimiter, final String header) - + @VisibleForTesting + DelimitedParser(final Optional delimiter, final Optional listDelimiter, final String header) { - this(delimiter, listDelimiter); + this(delimiter, listDelimiter, false, 0); setFieldNames(header); } @@ -106,6 +123,14 @@ public String getListDelimiter() return listDelimiter; } + @Override + public void startFileFromBeginning() + { + supportSkipHeaderRows = true; + hasParsedHeader = false; + skippedHeaderRows = 0; + } + @Override public List getFieldNames() { @@ -115,8 +140,10 @@ public List getFieldNames() @Override public void setFieldNames(final Iterable fieldNames) { - ParserUtils.validateFields(fieldNames); - this.fieldNames = Lists.newArrayList(fieldNames); + if (fieldNames != null) { + ParserUtils.validateFields(fieldNames); + this.fieldNames = Lists.newArrayList(fieldNames); + } } public void setFieldNames(String header) @@ -132,9 +159,27 @@ public void setFieldNames(String header) @Override public Map parse(final String input) { + if (!supportSkipHeaderRows && (hasHeaderRow || maxSkipHeaderRows > 0)) { + throw new UnsupportedOperationException("hasHeaderRow or maxSkipHeaderRows is not supported. " + + "Please check the indexTask supports these options."); + } + try { Iterable values = splitter.split(input); + if (skippedHeaderRows < maxSkipHeaderRows) { + skippedHeaderRows++; + return null; + } + + if (hasHeaderRow && !hasParsedHeader) { + if (fieldNames == null) { + setFieldNames(values); + } + hasParsedHeader = true; + return null; + } + if (fieldNames == null) { setFieldNames(ParserUtils.generateFieldNames(Iterators.size(values.iterator()))); } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java index 2717d2db915a..d23d599d3bca 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/JSONPathParser.java @@ -238,7 +238,7 @@ public enum FieldType /** * Specifies a field to be added to the parsed object Map, using JsonPath notation. - * + *

* See https://github.com/jayway/JsonPath for more information. */ public static class FieldSpec @@ -281,5 +281,4 @@ public String getExpr() return expr; } } - } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/JavaScriptParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/JavaScriptParser.java index 2357be2245c4..6cbb6e430516 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/JavaScriptParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/JavaScriptParser.java @@ -44,6 +44,7 @@ private static Function compile(String function) return new Function() { + @Override public Object apply(Object input) { // ideally we need a close() function to discard the context once it is not used anymore diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/Parser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/Parser.java index 78b29de72244..8cc6fd6a1d6b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/Parser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/Parser.java @@ -19,6 +19,7 @@ package io.druid.java.util.common.parsers; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -28,11 +29,21 @@ public interface Parser { /** - * Parse a String into a Map. + * This method may or may not get called at the start of reading of every file depending on the type of IndexTasks. + * The parser state should be reset if exists. + */ + default void startFileFromBeginning() + { + + } + + /** + * Parse a String into a Map. The result can be null which means the given input string will be ignored. * * @throws ParseException if the String cannot be parsed */ - public Map parse(String input); + @Nullable + Map parse(String input); /** * Set the fieldNames that you expect to see in parsed Maps. Deprecated; Parsers should not, in general, be @@ -40,12 +51,12 @@ public interface Parser * parser) and those parsers have their own way of setting field names. */ @Deprecated - public void setFieldNames(Iterable fieldNames); + void setFieldNames(Iterable fieldNames); /** * Returns the fieldNames that we expect to see in parsed Maps, if known, or null otherwise. Deprecated; Parsers * should not, in general, be expected to know what fields they will return. */ @Deprecated - public List getFieldNames(); + List getFieldNames(); } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java index 87b6321aa20f..329b02aa944a 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java @@ -117,7 +117,6 @@ public void setFieldNames(Iterable fieldNames) } @Override - public List getFieldNames() { return fieldNames; diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/ToLowerCaseParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/ToLowerCaseParser.java index 7e45bf23d941..fede1aa1f98c 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/ToLowerCaseParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/ToLowerCaseParser.java @@ -53,6 +53,12 @@ public Map parse(String input) return retVal; } + @Override + public void startFileFromBeginning() + { + baseParser.startFileFromBeginning(); + } + @Override public void setFieldNames(Iterable fieldNames) { diff --git a/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java b/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java index 4a9b65ca5797..292715b1b6d0 100644 --- a/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java @@ -422,7 +422,7 @@ public void testCustomNestedPeriodFail() } } - private class PathDate + private static class PathDate { public final String path; public final DateTime date; diff --git a/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java b/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java index 8121fd9fd0c7..37a589b276a2 100644 --- a/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java @@ -21,7 +21,7 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import java.util.Map; @@ -80,7 +80,7 @@ public void testCSVParserWithHeader() @Test public void testCSVParserWithoutHeader() { - final Parser csvParser = new CSVParser(Optional.fromNullable(null)); + final Parser csvParser = new CSVParser(Optional.fromNullable(null), false, 0); String body = "hello,world,foo"; final Map jsonMap = csvParser.parse(body); Assert.assertEquals( @@ -89,4 +89,48 @@ public void testCSVParserWithoutHeader() jsonMap ); } + + @Test + public void testCSVParserWithSkipHeaderRows() + { + final int skipHeaderRows = 2; + final Parser csvParser = new CSVParser( + Optional.absent(), + false, + skipHeaderRows + ); + csvParser.startFileFromBeginning(); + final String[] body = new String[] { + "header,line,1", + "header,line,2", + "hello,world,foo" + }; + int index; + for (index = 0; index < skipHeaderRows; index++) { + Assert.assertNull(csvParser.parse(body[index])); + } + final Map jsonMap = csvParser.parse(body[index]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), + jsonMap + ); + } + + @Test(expected = UnsupportedOperationException.class) + public void testCSVParserWithoutStartFileFromBeginning() + { + final int skipHeaderRows = 2; + final Parser csvParser = new CSVParser( + Optional.absent(), + false, + skipHeaderRows + ); + final String[] body = new String[] { + "header\tline\t1", + "header\tline\t2", + "hello\tworld\tfoo" + }; + csvParser.parse(body[0]); + } } diff --git a/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java b/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java index 3ca58c67f671..d91ed25cbbc2 100644 --- a/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java @@ -21,7 +21,7 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import java.util.Map; @@ -67,7 +67,11 @@ public void testInvalidHeader() public void testTSVParserWithHeader() { String header = "time\tvalue1\tvalue2"; - final Parser delimitedParser = new DelimitedParser(Optional.of("\t"), Optional.absent(), header); + final Parser delimitedParser = new DelimitedParser( + Optional.of("\t"), + Optional.absent(), + header + ); String body = "hello\tworld\tfoo"; final Map jsonMap = delimitedParser.parse(body); Assert.assertEquals( @@ -80,7 +84,12 @@ public void testTSVParserWithHeader() @Test public void testTSVParserWithoutHeader() { - final Parser delimitedParser = new DelimitedParser(Optional.of("\t"), Optional.absent()); + final Parser delimitedParser = new DelimitedParser( + Optional.of("\t"), + Optional.absent(), + false, + 0 + ); String body = "hello\tworld\tfoo"; final Map jsonMap = delimitedParser.parse(body); Assert.assertEquals( @@ -89,4 +98,50 @@ public void testTSVParserWithoutHeader() jsonMap ); } + + @Test + public void testTSVParserWithSkipHeaderRows() + { + final int skipHeaderRows = 2; + final Parser delimitedParser = new DelimitedParser( + Optional.of("\t"), + Optional.absent(), + false, + skipHeaderRows + ); + delimitedParser.startFileFromBeginning(); + final String[] body = new String[] { + "header\tline\t1", + "header\tline\t2", + "hello\tworld\tfoo" + }; + int index; + for (index = 0; index < skipHeaderRows; index++) { + Assert.assertNull(delimitedParser.parse(body[index])); + } + final Map jsonMap = delimitedParser.parse(body[index]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), + jsonMap + ); + } + + @Test(expected = UnsupportedOperationException.class) + public void testTSVParserWithoutStartFileFromBeginning() + { + final int skipHeaderRows = 2; + final Parser delimitedParser = new DelimitedParser( + Optional.of("\t"), + Optional.absent(), + false, + skipHeaderRows + ); + final String[] body = new String[] { + "header\tline\t1", + "header\tline\t2", + "hello\tworld\tfoo" + }; + delimitedParser.parse(body[0]); + } } diff --git a/pom.xml b/pom.xml index 873c4ecaa42c..47f2865bcac0 100644 --- a/pom.xml +++ b/pom.xml @@ -71,7 +71,7 @@ 4.1.6.Final 1.7.12 - 2.3.0 + 2.7.3 2.0.0 1.6.6 @@ -106,6 +106,7 @@ extensions-core/kafka-indexing-service extensions-core/mysql-metadata-storage extensions-core/postgresql-metadata-storage + extensions-core/protobuf-extensions extensions-core/lookups-cached-global extensions-core/lookups-cached-single extensions-core/s3-extensions @@ -129,6 +130,8 @@ extensions-contrib/scan-query extensions-contrib/sqlserver-metadata-storage extensions-contrib/kafka-emitter + + distribution @@ -177,7 +180,9 @@ com.amazonaws aws-java-sdk - 1.10.21 + + 1.10.56 javax.mail @@ -873,7 +878,7 @@ - -Xmx1024m -Duser.language=en -Duser.country=US -Dfile.encoding=UTF-8 + -Xmx3000m -Duser.language=en -Duser.country=US -Dfile.encoding=UTF-8 -Duser.timezone=UTC -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager @@ -993,8 +998,8 @@ org.apache.maven.plugins maven-compiler-plugin - 1.8 - 1.8 + 1.8 + 1.8 @@ -1005,38 +1010,71 @@ strict - - - - org.apache.maven.plugins - maven-compiler-plugin - - javac-with-errorprone - true - - - - org.codehaus.plexus - plexus-compiler-javac-errorprone - 2.5 - - - - - + + + org.apache.maven.plugins + maven-compiler-plugin + + javac-with-errorprone + true + true + 1024m + 3000m + 1.8 + 1.8 + false + + -XepDisableWarningsInGeneratedCode + + -Xep:ClassCanBeStatic:ERROR + -Xep:PreconditionsInvalidPlaceholder:ERROR + -Xep:MissingOverride:ERROR + + -Xep:ArgumentParameterSwap + -Xep:AssistedInjectAndInjectOnSameConstructor + -Xep:AutoFactoryAtInject + -Xep:ClassName + -Xep:ComparisonContractViolated + -Xep:DepAnn + -Xep:DivZero + -Xep:EmptyIf + -Xep:InjectInvalidTargetingOnScopingAnnotation + -Xep:InjectMoreThanOneQualifier + -Xep:InjectScopeAnnotationOnInterfaceOrAbstractClass + -Xep:InjectScopeOrQualifierAnnotationRetention + -Xep:InjectedConstructorAnnotations + -Xep:InsecureCryptoUsage + -Xep:JMockTestWithoutRunWithOrRuleAnnotation + -Xep:JavaxInjectOnFinalField + -Xep:LockMethodChecker + -Xep:LongLiteralLowerCaseSuffix + -Xep:NoAllocation + -Xep:NonRuntimeAnnotation + -Xep:NumericEquality + -Xep:ParameterPackage + -Xep:ProtoStringFieldReferenceEquality + -Xep:QualifierOnMethodWithoutProvides + -Xep:UnlockMethod + + + + + org.codehaus.plexus + plexus-compiler-javac-errorprone + 2.8.1 + + + + com.google.errorprone + error_prone_core + 2.0.19 + + + + - - java8 - - 1.8 - - - - distribution - - - parallel-test diff --git a/processing/pom.xml b/processing/pom.xml index ee9564ebb202..9d95e413dc05 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -65,10 +65,6 @@ org.skife.config config-magic - - com.google.protobuf - protobuf-java - commons-io commons-io @@ -132,6 +128,22 @@ + + org.apache.maven.plugins + maven-compiler-plugin + + + + io.druid + druid-common + ${project.parent.version} + + + + io.druid.annotations.SubclassesMustBePublicAnnotationProcessor + + + org.apache.maven.plugins maven-jar-plugin diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java deleted file mode 100644 index 557ca0941259..000000000000 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.data.input; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; -import com.google.protobuf.ByteString; -import com.google.protobuf.DescriptorProtos; -import com.google.protobuf.Descriptors; -import com.google.protobuf.DynamicMessage; -import com.google.protobuf.InvalidProtocolBufferException; -import io.druid.data.input.impl.MapInputRowParser; -import io.druid.data.input.impl.ParseSpec; -import io.druid.java.util.common.logger.Logger; - -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.util.Map; - -@JsonTypeName("protobuf") -public class ProtoBufInputRowParser implements ByteBufferInputRowParser -{ - private static final Logger log = new Logger(ProtoBufInputRowParser.class); - - private final ParseSpec parseSpec; - private final MapInputRowParser mapParser; - private final String descriptorFileInClasspath; - - @JsonCreator - public ProtoBufInputRowParser( - @JsonProperty("parseSpec") ParseSpec parseSpec, - @JsonProperty("descriptor") String descriptorFileInClasspath - ) - { - this.parseSpec = parseSpec; - this.descriptorFileInClasspath = descriptorFileInClasspath; - this.mapParser = new MapInputRowParser(this.parseSpec); - } - - @Override - public ParseSpec getParseSpec() - { - return parseSpec; - } - - @Override - public ProtoBufInputRowParser withParseSpec(ParseSpec parseSpec) - { - return new ProtoBufInputRowParser(parseSpec, descriptorFileInClasspath); - } - - @Override - public InputRow parse(ByteBuffer input) - { - // We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses - // the DynamicMessage directly... - Map theMap = buildStringKeyMap(input); - - return mapParser.parse(theMap); - } - - private Map buildStringKeyMap(ByteBuffer input) - { - final Descriptors.Descriptor descriptor = getDescriptor(descriptorFileInClasspath); - final Map theMap = Maps.newHashMap(); - - try { - DynamicMessage message = DynamicMessage.parseFrom(descriptor, ByteString.copyFrom(input)); - Map allFields = message.getAllFields(); - - for (Map.Entry entry : allFields.entrySet()) { - String name = entry.getKey().getName(); - if (theMap.containsKey(name)) { - continue; - // Perhaps throw an exception here? - // throw new RuntimeException("dupicate key " + name + " in " + message); - } - Object value = entry.getValue(); - if (value instanceof Descriptors.EnumValueDescriptor) { - Descriptors.EnumValueDescriptor desc = (Descriptors.EnumValueDescriptor) value; - value = desc.getName(); - } - - theMap.put(name, value); - } - - } - catch (InvalidProtocolBufferException e) { - log.warn(e, "Problem with protobuf something"); - } - return theMap; - } - - private Descriptors.Descriptor getDescriptor(String descriptorFileInClassPath) - { - try { - InputStream fin = this.getClass().getClassLoader().getResourceAsStream(descriptorFileInClassPath); - DescriptorProtos.FileDescriptorSet set = DescriptorProtos.FileDescriptorSet.parseFrom(fin); - Descriptors.FileDescriptor file = Descriptors.FileDescriptor.buildFrom( - set.getFile(0), new Descriptors.FileDescriptor[] - {} - ); - return file.getMessageTypes().get(0); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } -} diff --git a/processing/src/main/java/io/druid/guice/ExtensionsConfig.java b/processing/src/main/java/io/druid/guice/ExtensionsConfig.java index 3b6b595919ca..e04e612a53d5 100644 --- a/processing/src/main/java/io/druid/guice/ExtensionsConfig.java +++ b/processing/src/main/java/io/druid/guice/ExtensionsConfig.java @@ -41,6 +41,10 @@ public class ExtensionsConfig @JsonProperty private String hadoopContainerDruidClasspath = null; + //Only applicable when hadoopContainerDruidClasspath is explicitly specified. + @JsonProperty + private boolean addExtensionsToHadoopContainer = false; + @JsonProperty private List loadList; @@ -64,6 +68,11 @@ public String getHadoopContainerDruidClasspath() return hadoopContainerDruidClasspath; } + public boolean getAddExtensionsToHadoopContainer() + { + return addExtensionsToHadoopContainer; + } + public List getLoadList() { return loadList; @@ -77,6 +86,7 @@ public String toString() ", directory='" + directory + '\'' + ", hadoopDependenciesDir='" + hadoopDependenciesDir + '\'' + ", hadoopContainerDruidClasspath='" + hadoopContainerDruidClasspath + '\'' + + ", addExtensionsToHadoopContainer=" + addExtensionsToHadoopContainer + ", loadList=" + loadList + '}'; } diff --git a/processing/src/main/java/io/druid/query/AsyncQueryRunner.java b/processing/src/main/java/io/druid/query/AsyncQueryRunner.java index ce5f1fc64b1b..182d18b1f0d8 100644 --- a/processing/src/main/java/io/druid/query/AsyncQueryRunner.java +++ b/processing/src/main/java/io/druid/query/AsyncQueryRunner.java @@ -47,9 +47,11 @@ public AsyncQueryRunner(QueryRunner baseRunner, ExecutorService executor, Que } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { + final Query query = queryPlus.getQuery(); final int priority = QueryContexts.getPriority(query); + final QueryPlus threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState(); final ListenableFuture> future = executor.submit(new AbstractPrioritizedCallable>(priority) { @Override @@ -57,7 +59,7 @@ public Sequence call() throws Exception { //Note: this is assumed that baseRunner does most of the work eagerly on call to the //run() method and resulting sequence accumulate/yield is fast. - return baseRunner.run(query, responseContext); + return baseRunner.run(threadSafeQueryPlus, responseContext); } }); queryWatcher.registerQuery(query, future); diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 7900474505f5..4a3e4716a592 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -92,6 +92,7 @@ public Sequence run(QuerySegmentWalker walker, Map context) return run(querySegmentSpec.lookup(this, walker), context); } + @Override public Sequence run(QueryRunner runner, Map context) { return runner.run(this, context); @@ -145,6 +146,16 @@ public boolean getContextBoolean(String key, boolean defaultValue) return QueryContexts.parseBoolean(this, key, defaultValue); } + /** + * @deprecated use {@link #computeOverriddenContext(Map, Map) computeOverriddenContext(getContext(), overrides))} + * instead. This method may be removed in the next minor or major version of Druid. + */ + @Deprecated + protected Map computeOverridenContext(final Map overrides) + { + return computeOverriddenContext(getContext(), overrides); + } + protected static Map computeOverriddenContext( final Map context, final Map overrides diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index e57b9471ce3b..bea2a3279a50 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -24,7 +24,7 @@ import io.druid.java.util.common.guava.Sequences; import org.joda.time.DateTime; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -49,24 +49,24 @@ public BySegmentQueryRunner( @Override @SuppressWarnings("unchecked") - public Sequence run(final Query query, Map responseContext) + public Sequence run(final QueryPlus queryPlus, Map responseContext) { - if (QueryContexts.isBySegment(query)) { - final Sequence baseSequence = base.run(query, responseContext); + if (QueryContexts.isBySegment(queryPlus.getQuery())) { + final Sequence baseSequence = base.run(queryPlus, responseContext); final List results = Sequences.toList(baseSequence, Lists.newArrayList()); return Sequences.simple( - Arrays.asList( + Collections.singletonList( (T) new Result>( timestamp, new BySegmentResultValueClass( results, segmentIdentifier, - query.getIntervals().get(0) + queryPlus.getQuery().getIntervals().get(0) ) ) ) ); } - return base.run(query, responseContext); + return base.run(queryPlus, responseContext); } } diff --git a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java index 373c2e3b1192..5dda6f618a3c 100644 --- a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -38,14 +38,14 @@ public BySegmentSkippingQueryRunner( } @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - if (QueryContexts.isBySegment(query)) { - return baseRunner.run(query, responseContext); + if (QueryContexts.isBySegment(queryPlus.getQuery())) { + return baseRunner.run(queryPlus, responseContext); } - return doRun(baseRunner, query, responseContext); + return doRun(baseRunner, queryPlus, responseContext); } - protected abstract Sequence doRun(QueryRunner baseRunner, Query query, Map context); + protected abstract Sequence doRun(QueryRunner baseRunner, QueryPlus queryPlus, Map context); } diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index 0d42ab983d4a..5f90615fba4a 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -33,14 +33,14 @@ public class CPUTimeMetricQueryRunner implements QueryRunner { private final QueryRunner delegate; - private final QueryToolChest> queryToolChest; + private final QueryToolChest> queryToolChest; private final ServiceEmitter emitter; private final AtomicLong cpuTimeAccumulator; private final boolean report; private CPUTimeMetricQueryRunner( QueryRunner delegate, - QueryToolChest> queryToolChest, + QueryToolChest> queryToolChest, ServiceEmitter emitter, AtomicLong cpuTimeAccumulator, boolean report @@ -58,9 +58,10 @@ private CPUTimeMetricQueryRunner( @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - final Sequence baseSequence = delegate.run(query, responseContext); + final QueryPlus queryWithMetrics = queryPlus.withQueryMetrics(queryToolChest); + final Sequence baseSequence = delegate.run(queryWithMetrics, responseContext); return Sequences.wrap( baseSequence, new SequenceWrapper() @@ -82,16 +83,17 @@ public void after(boolean isDone, Throwable thrown) throws Exception if (report) { final long cpuTimeNs = cpuTimeAccumulator.get(); if (cpuTimeNs > 0) { - queryToolChest.makeMetrics(query).reportCpuTime(cpuTimeNs).emit(emitter); + queryWithMetrics.getQueryMetrics().reportCpuTime(cpuTimeNs).emit(emitter); } } } } ); } + public static QueryRunner safeBuild( QueryRunner delegate, - QueryToolChest> queryToolChest, + QueryToolChest> queryToolChest, ServiceEmitter emitter, AtomicLong accumulator, boolean report diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 117d6436082b..1312fb1b024e 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -89,11 +89,12 @@ public ChainedExecutionQueryRunner( } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { + Query query = queryPlus.getQuery(); final int priority = QueryContexts.getPriority(query); final Ordering ordering = query.getResultOrdering(); - + final QueryPlus threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState(); return new BaseSequence>( new BaseSequence.IteratorMaker>() { @@ -121,7 +122,7 @@ public ListenableFuture> apply(final QueryRunner input) public Iterable call() throws Exception { try { - Sequence result = input.run(query, responseContext); + Sequence result = input.run(threadSafeQueryPlus, responseContext); if (result == null) { throw new ISE("Got a null result! Segments are missing!"); } @@ -132,11 +133,9 @@ public Iterable call() throws Exception } return retVal; - } - catch (QueryInterruptedException e) { + } catch (QueryInterruptedException e) { throw Throwables.propagate(e); - } - catch (Exception e) { + } catch (Exception e) { log.error(e, "Exception with one of the sequences!"); throw Throwables.propagate(e); } @@ -155,24 +154,20 @@ public Iterable call() throws Exception return new MergeIterable<>( ordering.nullsFirst(), QueryContexts.hasTimeout(query) ? - futures.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS) : - futures.get() + futures.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS) : + futures.get() ).iterator(); - } - catch (InterruptedException e) { + } catch (InterruptedException e) { log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); futures.cancel(true); throw new QueryInterruptedException(e); - } - catch (CancellationException e) { + } catch (CancellationException e) { throw new QueryInterruptedException(e); - } - catch (TimeoutException e) { + } catch (TimeoutException e) { log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); futures.cancel(true); throw new QueryInterruptedException(e); - } - catch (ExecutionException e) { + } catch (ExecutionException e) { throw Throwables.propagate(e.getCause()); } } diff --git a/processing/src/main/java/io/druid/query/ConcatQueryRunner.java b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java index e36ec4cdc2d1..2237ad8c5b1a 100644 --- a/processing/src/main/java/io/druid/query/ConcatQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java @@ -38,7 +38,7 @@ public ConcatQueryRunner( } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { return Sequences.concat( Sequences.map( @@ -48,7 +48,7 @@ public Sequence run(final Query query, final Map responseC @Override public Sequence apply(final QueryRunner input) { - return input.run(query, responseContext); + return input.run(queryPlus, responseContext); } } ) diff --git a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java index b71da6e4dbb0..3b03960196f9 100644 --- a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java @@ -22,8 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; +import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; +@LazySingleton public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory { private static final GenericQueryMetricsFactory INSTANCE = @@ -43,7 +46,7 @@ public static GenericQueryMetricsFactory instance() private final ObjectMapper jsonMapper; @Inject - public DefaultGenericQueryMetricsFactory(ObjectMapper jsonMapper) + public DefaultGenericQueryMetricsFactory(@Json ObjectMapper jsonMapper) { this.jsonMapper = jsonMapper; } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index a19053be8789..57c6099f23bd 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -31,17 +31,42 @@ import java.util.Map; import java.util.concurrent.TimeUnit; +/** + * DefaultQueryMetrics is unsafe for use from multiple threads. It fails with RuntimeException on access not from the + * thread where it was constructed. To "transfer" DefaultQueryMetrics from one thread to another {@link #ownerThread} + * field should be updated. + */ public class DefaultQueryMetrics> implements QueryMetrics { protected final ObjectMapper jsonMapper; protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); protected final Map metrics = new HashMap<>(); + /** Non final to give subclasses ability to reassign it. */ + protected Thread ownerThread = Thread.currentThread(); + public DefaultQueryMetrics(ObjectMapper jsonMapper) { this.jsonMapper = jsonMapper; } + protected void checkModifiedFromOwnerThread() + { + if (Thread.currentThread() != ownerThread) { + throw new IllegalStateException( + "DefaultQueryMetrics must not be modified from multiple threads. If it is needed to gather dimension or " + + "metric information from multiple threads or from an async thread, this information should explicitly be " + + "passed between threads (e. g. using Futures), or this DefaultQueryMetrics's ownerThread should be " + + "reassigned explicitly"); + } + } + + protected void setDimension(String dimension, String value) + { + checkModifiedFromOwnerThread(); + builder.setDimension(dimension, value); + } + @Override public void query(QueryType query) { @@ -56,18 +81,19 @@ public void query(QueryType query) @Override public void dataSource(QueryType query) { - builder.setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())); + setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())); } @Override public void queryType(QueryType query) { - builder.setDimension(DruidMetrics.TYPE, query.getType()); + setDimension(DruidMetrics.TYPE, query.getType()); } @Override public void interval(QueryType query) { + checkModifiedFromOwnerThread(); builder.setDimension( DruidMetrics.INTERVAL, query.getIntervals().stream().map(Interval::toString).toArray(String[]::new) @@ -77,32 +103,28 @@ public void interval(QueryType query) @Override public void hasFilters(QueryType query) { - builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); + setDimension("hasFilters", String.valueOf(query.hasFilters())); } @Override public void duration(QueryType query) { - builder.setDimension("duration", query.getDuration().toString()); + setDimension("duration", query.getDuration().toString()); } @Override public void queryId(QueryType query) { - builder.setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); + setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); } @Override public void context(QueryType query) { try { - builder.setDimension( + setDimension( "context", - jsonMapper.writeValueAsString( - query.getContext() == null - ? ImmutableMap.of() - : query.getContext() - ) + jsonMapper.writeValueAsString(query.getContext() == null ? ImmutableMap.of() : query.getContext()) ); } catch (JsonProcessingException e) { @@ -113,111 +135,115 @@ public void context(QueryType query) @Override public void server(String host) { - builder.setDimension("server", host); + setDimension("server", host); } @Override public void remoteAddress(String remoteAddress) { - builder.setDimension("remoteAddress", remoteAddress); + setDimension("remoteAddress", remoteAddress); } @Override public void status(String status) { - builder.setDimension(DruidMetrics.STATUS, status); + setDimension(DruidMetrics.STATUS, status); } @Override public void success(boolean success) { - builder.setDimension("success", String.valueOf(success)); + setDimension("success", String.valueOf(success)); } @Override public void segment(String segmentIdentifier) { - builder.setDimension("segment", segmentIdentifier); + setDimension("segment", segmentIdentifier); } @Override public void chunkInterval(Interval interval) { - builder.setDimension("chunkInterval", interval.toString()); + setDimension("chunkInterval", interval.toString()); } @Override public QueryMetrics reportQueryTime(long timeNs) { - return defaultTimeMetric("query/time", timeNs); + return reportMillisTimeMetric("query/time", timeNs); } @Override public QueryMetrics reportQueryBytes(long byteCount) { - metrics.put("query/bytes", byteCount); - return this; + return reportMetric("query/bytes", byteCount); } @Override public QueryMetrics reportWaitTime(long timeNs) { - return defaultTimeMetric("query/wait/time", timeNs); + return reportMillisTimeMetric("query/wait/time", timeNs); } @Override public QueryMetrics reportSegmentTime(long timeNs) { - return defaultTimeMetric("query/segment/time", timeNs); + return reportMillisTimeMetric("query/segment/time", timeNs); } @Override public QueryMetrics reportSegmentAndCacheTime(long timeNs) { - return defaultTimeMetric("query/segmentAndCache/time", timeNs); + return reportMillisTimeMetric("query/segmentAndCache/time", timeNs); } @Override public QueryMetrics reportIntervalChunkTime(long timeNs) { - return defaultTimeMetric("query/intervalChunk/time", timeNs); + return reportMillisTimeMetric("query/intervalChunk/time", timeNs); } @Override public QueryMetrics reportCpuTime(long timeNs) { - metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); - return this; + return reportMetric("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); } @Override public QueryMetrics reportNodeTimeToFirstByte(long timeNs) { - return defaultTimeMetric("query/node/ttfb", timeNs); + return reportMillisTimeMetric("query/node/ttfb", timeNs); } @Override public QueryMetrics reportNodeTime(long timeNs) { - return defaultTimeMetric("query/node/time", timeNs); + return reportMillisTimeMetric("query/node/time", timeNs); } - private QueryMetrics defaultTimeMetric(String metricName, long timeNs) + private QueryMetrics reportMillisTimeMetric(String metricName, long timeNs) { - metrics.put(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); + return reportMetric(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); + } + + protected QueryMetrics reportMetric(String metricName, Number value) + { + checkModifiedFromOwnerThread(); + metrics.put(metricName, value); return this; } @Override public QueryMetrics reportNodeBytes(long byteCount) { - metrics.put("query/node/bytes", byteCount); - return this; + return reportMetric("query/node/bytes", byteCount); } @Override public void emit(ServiceEmitter emitter) { + checkModifiedFromOwnerThread(); for (Map.Entry metric : metrics.entrySet()) { emitter.emit(builder.build(metric.getKey(), metric.getValue())); } diff --git a/processing/src/main/java/io/druid/query/DruidProcessingConfig.java b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java index 94d349fd3fb7..5445e5a7324b 100644 --- a/processing/src/main/java/io/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java @@ -52,6 +52,7 @@ public int getNumMergeBuffers() return Math.max(2, getNumThreads() / 4); } + @Override @Config(value = "${base_path}.columnCache.sizeBytes") public int columnCacheSizeBytes() { diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index 7fe58ee06b90..acf52b2aea46 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -47,8 +47,9 @@ public FinalizeResultsQueryRunner( } @Override - public Sequence run(final Query query, Map responseContext) + public Sequence run(final QueryPlus queryPlus, Map responseContext) { + final Query query = queryPlus.getQuery(); final boolean isBySegment = QueryContexts.isBySegment(query); final boolean shouldFinalize = QueryContexts.isFinalize(query, true); @@ -100,7 +101,7 @@ public T apply(T input) return Sequences.map( - baseRunner.run(queryToRun, responseContext), + baseRunner.run(queryPlus.withQuery(queryToRun), responseContext), finalizerFn ); diff --git a/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java b/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java index 71bf44451cda..a52da78c9ef5 100644 --- a/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java +++ b/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java @@ -49,10 +49,10 @@ public FluentQueryRunner(QueryRunner runner) @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { - return baseRunner.run(query, responseContext); + return baseRunner.run(queryPlus, responseContext); } public FluentQueryRunner from(QueryRunner runner) { diff --git a/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java index dd4297606da0..5882d9c79222 100644 --- a/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java @@ -22,6 +22,16 @@ /** * This factory is used for DI of custom {@link QueryMetrics} implementations for all query types, which don't (yet) * need to emit custom dimensions and/or metrics, i. e. they are good with the generic {@link QueryMetrics} interface. + * + * Implementations could be injected using + * + * PolyBind + * .optionBinder(binder, Key.get(GenericQueryMetricsFactory.class)) + * .addBinding("myCustomGenericQueryMetricsFactory") + * .to(MyCustomGenericQueryMetricsFactory.class); + * + * And then setting property: + * druid.query.generic.queryMetricsFactory=myCustomGenericQueryMetricsFactory */ public interface GenericQueryMetricsFactory { diff --git a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java index 77775f295c6f..dda375b100be 100644 --- a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java @@ -78,9 +78,9 @@ public GroupByMergedQueryRunner( } @Override - public Sequence run(final Query queryParam, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - final GroupByQuery query = (GroupByQuery) queryParam; + final GroupByQuery query = (GroupByQuery) queryPlus.getQuery(); final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query); final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded(); final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( @@ -92,8 +92,8 @@ public Sequence run(final Query queryParam, final Map resp final Pair> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair(); final boolean bySegment = QueryContexts.isBySegment(query); final int priority = QueryContexts.getPriority(query); - - ListenableFuture> futures = Futures.allAsList( + final QueryPlus threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState(); + final ListenableFuture> futures = Futures.allAsList( Lists.newArrayList( Iterables.transform( queryables, @@ -114,10 +114,10 @@ public Void call() throws Exception { try { if (bySegment) { - input.run(queryParam, responseContext) + input.run(threadSafeQueryPlus, responseContext) .accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs); } else { - input.run(queryParam, responseContext) + input.run(threadSafeQueryPlus, responseContext) .accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); } diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 5e8b529b1535..f4d3467fb315 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -31,7 +31,7 @@ import org.joda.time.Interval; import org.joda.time.Period; -import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -63,18 +63,18 @@ public IntervalChunkingQueryRunner( } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - final Period chunkPeriod = getChunkPeriod(query); + final Period chunkPeriod = getChunkPeriod(queryPlus.getQuery()); // Check for non-empty chunkPeriod, avoiding toStandardDuration since that cannot handle periods like P1M. if (EPOCH.plus(chunkPeriod).getMillis() == EPOCH.getMillis()) { - return baseRunner.run(query, responseContext); + return baseRunner.run(queryPlus, responseContext); } List chunkIntervals = Lists.newArrayList( FunctionalIterable - .create(query.getIntervals()) + .create(queryPlus.getQuery().getIntervals()) .transformCat( new Function>() { @@ -88,7 +88,7 @@ public Iterable apply(Interval input) ); if (chunkIntervals.size() <= 1) { - return baseRunner.run(query, responseContext); + return baseRunner.run(queryPlus, responseContext); } return Sequences.concat( @@ -113,7 +113,8 @@ public Sequence apply(Interval singleInterval) ), executor, queryWatcher ).run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))), + queryPlus.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(singleInterval))), responseContext ); } diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java b/processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java index a45814cee329..9fbcec1e47a3 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java @@ -48,6 +48,7 @@ protected ListeningExecutorService delegate() return delegate; } + @SuppressWarnings("ParameterPackage") @Override public ListenableFuture submit(Callable tCallable) { diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 1f4041973b4e..69d1db8094c2 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -34,19 +34,19 @@ public class MetricsEmittingQueryRunner implements QueryRunner { private final ServiceEmitter emitter; - private final QueryToolChest> queryToolChest; + private final QueryToolChest> queryToolChest; private final QueryRunner queryRunner; private final long creationTimeNs; - private final ObjLongConsumer>> reportMetric; - private final Consumer>> applyCustomDimensions; + private final ObjLongConsumer> reportMetric; + private final Consumer> applyCustomDimensions; private MetricsEmittingQueryRunner( ServiceEmitter emitter, - QueryToolChest> queryToolChest, + QueryToolChest> queryToolChest, QueryRunner queryRunner, long creationTimeNs, - ObjLongConsumer>> reportMetric, - Consumer>> applyCustomDimensions + ObjLongConsumer> reportMetric, + Consumer> applyCustomDimensions ) { this.emitter = emitter; @@ -59,10 +59,10 @@ private MetricsEmittingQueryRunner( public MetricsEmittingQueryRunner( ServiceEmitter emitter, - QueryToolChest> queryToolChest, + QueryToolChest> queryToolChest, QueryRunner queryRunner, - ObjLongConsumer>> reportMetric, - Consumer>> applyCustomDimensions + ObjLongConsumer> reportMetric, + Consumer> applyCustomDimensions ) { this(emitter, queryToolChest, queryRunner, -1, reportMetric, applyCustomDimensions); @@ -81,9 +81,10 @@ public MetricsEmittingQueryRunner withWaitMeasuredFromNow() } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - final QueryMetrics> queryMetrics = queryToolChest.makeMetrics(query); + QueryPlus queryWithMetrics = queryPlus.withQueryMetrics(queryToolChest); + final QueryMetrics queryMetrics = queryWithMetrics.getQueryMetrics(); applyCustomDimensions.accept(queryMetrics); @@ -91,7 +92,7 @@ public Sequence run(final Query query, final Map responseC // Use LazySequence because want to account execution time of queryRunner.run() (it prepares the underlying // Sequence) as part of the reported query time, i. e. we want to execute queryRunner.run() after // `startTime = System.nanoTime();` (see below). - new LazySequence<>(() -> queryRunner.run(query, responseContext)), + new LazySequence<>(() -> queryRunner.run(queryWithMetrics, responseContext)), new SequenceWrapper() { private long startTimeNs; diff --git a/processing/src/main/java/io/druid/query/NoopQueryRunner.java b/processing/src/main/java/io/druid/query/NoopQueryRunner.java index 3d3aa19b30d2..b058a4d89399 100644 --- a/processing/src/main/java/io/druid/query/NoopQueryRunner.java +++ b/processing/src/main/java/io/druid/query/NoopQueryRunner.java @@ -29,7 +29,7 @@ public class NoopQueryRunner implements QueryRunner { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return Sequences.empty(); } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 9ad178161ead..4e6745ca6d43 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -70,8 +70,19 @@ public interface Query String getType(); + /** + * @deprecated use {@link QueryPlus#run(QuerySegmentWalker, Map)} instead. This method is going to be removed in Druid + * 0.11. In the future, a method like getRunner(QuerySegmentWalker, Map) could be added instead of this method, so + * that {@link QueryPlus#run(QuerySegmentWalker, Map)} could be implemented as {@code + * this.query.getRunner(walker, context).run(this, context))}. + */ + @Deprecated Sequence run(QuerySegmentWalker walker, Map context); + /** + * @deprecated use {@link QueryRunner#run(QueryPlus, Map)} instead. This method is going to be removed in Druid 0.11. + */ + @Deprecated Sequence run(QueryRunner runner, Map context); List getIntervals(); diff --git a/processing/src/main/java/io/druid/query/QueryContexts.java b/processing/src/main/java/io/druid/query/QueryContexts.java index 5a0409e430d4..473c3c50318d 100644 --- a/processing/src/main/java/io/druid/query/QueryContexts.java +++ b/processing/src/main/java/io/druid/query/QueryContexts.java @@ -21,12 +21,14 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; public class QueryContexts { public static final String PRIORITY_KEY = "priority"; public static final String TIMEOUT_KEY = "timeout"; + public static final String MAX_SCATTER_GATHER_BYTES_KEY = "maxScatterGatherBytes"; public static final String DEFAULT_TIMEOUT_KEY = "defaultTimeout"; public static final String CHUNK_PERIOD_KEY = "chunkPeriod"; @@ -98,6 +100,31 @@ public static String getChunkPeriod(Query query) return query.getContextValue(CHUNK_PERIOD_KEY, "P0D"); } + public static Query withMaxScatterGatherBytes(Query query, long maxScatterGatherBytesLimit) + { + Object obj = query.getContextValue(MAX_SCATTER_GATHER_BYTES_KEY); + if (obj == null) { + return query.withOverriddenContext(ImmutableMap.of(MAX_SCATTER_GATHER_BYTES_KEY, maxScatterGatherBytesLimit)); + } else { + long curr = ((Number) obj).longValue(); + if (curr > maxScatterGatherBytesLimit) { + throw new IAE( + "configured [%s = %s] is more than enforced limit of [%s].", + MAX_SCATTER_GATHER_BYTES_KEY, + curr, + maxScatterGatherBytesLimit + ); + } else { + return query; + } + } + } + + public static long getMaxScatterGatherBytes(Query query) + { + return parseLong(query, MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE); + } + public static boolean hasTimeout(Query query) { return getTimeout(query) != NO_TIMEOUT; @@ -111,10 +138,15 @@ public static long getTimeout(Query query) public static long getTimeout(Query query, long defaultTimeout) { final long timeout = parseLong(query, TIMEOUT_KEY, defaultTimeout); - Preconditions.checkState(timeout >= 0, "Timeout must be a non negative value, but was [%d]", timeout); + Preconditions.checkState(timeout >= 0, "Timeout must be a non negative value, but was [%s]", timeout); return timeout; } + public static Query withTimeout(Query query, long timeout) + { + return query.withOverriddenContext(ImmutableMap.of(TIMEOUT_KEY, timeout)); + } + public static Query withDefaultTimeout(Query query, long defaultTimeout) { return query.withOverriddenContext(ImmutableMap.of(QueryContexts.DEFAULT_TIMEOUT_KEY, defaultTimeout)); @@ -123,7 +155,7 @@ public static Query withDefaultTimeout(Query query, long defaultTimeou static long getDefaultTimeout(Query query) { final long defaultTimeout = parseLong(query, DEFAULT_TIMEOUT_KEY, DEFAULT_TIMEOUT_MILLIS); - Preconditions.checkState(defaultTimeout >= 0, "Timeout must be a non negative value, but was [%d]", defaultTimeout); + Preconditions.checkState(defaultTimeout >= 0, "Timeout must be a non negative value, but was [%s]", defaultTimeout); return defaultTimeout; } diff --git a/processing/src/main/java/io/druid/query/QueryDataSource.java b/processing/src/main/java/io/druid/query/QueryDataSource.java index 0797f2f5f807..6e96eaa65059 100644 --- a/processing/src/main/java/io/druid/query/QueryDataSource.java +++ b/processing/src/main/java/io/druid/query/QueryDataSource.java @@ -49,6 +49,7 @@ public Query getQuery() return query; } + @Override public String toString() { return query.toString(); } @Override diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 3afc3a122cb8..a184de96a54c 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -122,9 +122,13 @@ * 5. Inject and use SearchQueryMetricsFactory instead of {@link GenericQueryMetricsFactory} in {@link * io.druid.query.search.SearchQueryQueryToolChest}. * - * 6. Specify `binder.bind(SearchQueryMetricsFactory.class).to(DefaultSearchQueryMetricsFactory.class)` in - * QueryToolChestModule (if the query type belongs to the core druid-processing, e. g. SearchQuery) or in some - * extension-specific Guice module otherwise, if the query type is defined in an extension, e. g. ScanQuery. + * 6. Establish injection of SearchQueryMetricsFactory using config and provider method in QueryToolChestModule + * (see how it is done in QueryToolChestModule for existing query types with custom metrics, e. g. {@link + * io.druid.query.topn.TopNQueryMetricsFactory}), if the query type belongs to the core druid-processing, e. g. + * SearchQuery. If the query type defined in an extension, you can specify + * `binder.bind(ScanQueryMetricsFactory.class).to(DefaultScanQueryMetricsFactory.class)` in the extension's + * Guice module, if the query type is defined in an extension, e. g. ScanQuery. Or establish similar configuration, + * as for the core query types. * * This complex procedure is needed to ensure custom {@link GenericQueryMetricsFactory} specified by users still works * for the query type when query type decides to create their custom QueryMetrics subclass. diff --git a/processing/src/main/java/io/druid/query/QueryPlus.java b/processing/src/main/java/io/druid/query/QueryPlus.java new file mode 100644 index 000000000000..f6453068bd82 --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryPlus.java @@ -0,0 +1,134 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query; + +import com.google.common.base.Preconditions; +import io.druid.java.util.common.guava.Sequence; +import io.druid.query.spec.QuerySegmentSpec; + +import javax.annotation.Nullable; +import java.util.Map; + +/** + * An immutable composite object of {@link Query} + extra stuff needed in {@link QueryRunner}s. This "extra stuff" + * is only {@link QueryMetrics} yet. + */ +public final class QueryPlus +{ + /** + * Returns the minimum bare QueryPlus object with the given query. {@link #getQueryMetrics()} of the QueryPlus object, + * returned from this factory method, returns {@code null}. + */ + public static QueryPlus wrap(Query query) + { + Preconditions.checkNotNull(query); + return new QueryPlus<>(query, null); + } + + private final Query query; + private final QueryMetrics queryMetrics; + + private QueryPlus(Query query, QueryMetrics queryMetrics) + { + this.query = query; + this.queryMetrics = queryMetrics; + } + + public Query getQuery() + { + return query; + } + + @Nullable + public QueryMetrics getQueryMetrics() + { + return queryMetrics; + } + + /** + * Returns the same QueryPlus object, if it already has {@link QueryMetrics} ({@link #getQueryMetrics()} returns not + * null), or returns a new QueryPlus object with {@link Query} from this QueryPlus and QueryMetrics created using the + * given {@link QueryToolChest}, via {@link QueryToolChest#makeMetrics(Query)} method. + * + * By convention, callers of {@code withQueryMetrics()} must also call .getQueryMetrics().emit() on the returned + * QueryMetrics object, regardless if this object is the same as the object on which .withQueryMetrics() was initially + * called (i. e. it already had non-null QueryMetrics), or if it is a new QueryPlus object. See {@link + * MetricsEmittingQueryRunner} for example. + */ + public QueryPlus withQueryMetrics(QueryToolChest> queryToolChest) + { + if (queryMetrics != null) { + return this; + } else { + return new QueryPlus<>(query, ((QueryToolChest) queryToolChest).makeMetrics(query)); + } + } + + /** + * Returns a QueryPlus object without the components which are unsafe for concurrent use from multiple threads, + * therefore couldn't be passed down in concurrent or async {@link QueryRunner}s. + * + * Currently the only unsafe component is {@link QueryMetrics}, i. e. {@code withoutThreadUnsafeState()} call is + * equivalent to {@link #withoutQueryMetrics()}. + */ + public QueryPlus withoutThreadUnsafeState() + { + return withoutQueryMetrics(); + } + + /** + * Returns the same QueryPlus object, if it doesn't have {@link QueryMetrics} ({@link #getQueryMetrics()} returns + * null), or returns a new QueryPlus object with {@link Query} from this QueryPlus and null as QueryMetrics. + */ + private QueryPlus withoutQueryMetrics() + { + if (queryMetrics == null) { + return this; + } else { + return new QueryPlus<>(query, null); + } + } + + /** + * Equivalent of withQuery(getQuery().withQuerySegmentSpec(spec)). + */ + public QueryPlus withQuerySegmentSpec(QuerySegmentSpec spec) + { + return new QueryPlus<>(query.withQuerySegmentSpec(spec), queryMetrics); + } + + /** + * Returns a QueryPlus object with {@link QueryMetrics} from this QueryPlus object, and the provided {@link Query}. + */ + public QueryPlus withQuery(Query replacementQuery) + { + return new QueryPlus<>(replacementQuery, queryMetrics); + } + + public Sequence run(QuerySegmentWalker walker, Map context) + { + if (query instanceof BaseQuery) { + return ((BaseQuery) query).getQuerySegmentSpec().lookup(query, walker).run(this, context); + } else { + // fallback + return query.run(walker, context); + } + } +} diff --git a/processing/src/main/java/io/druid/query/QueryRunner.java b/processing/src/main/java/io/druid/query/QueryRunner.java index 58cdf9d70813..a7e17f43d242 100644 --- a/processing/src/main/java/io/druid/query/QueryRunner.java +++ b/processing/src/main/java/io/druid/query/QueryRunner.java @@ -24,14 +24,26 @@ import java.util.Map; /** + * This interface has two similar run() methods. {@link #run(Query, Map)} is legacy and {@link #run(QueryPlus, Map)} + * is the new one. Their default implementations delegate to each other. Every implementation of QueryRunner should + * override only one of those methods. New implementations should override the new method: {@link #run(QueryPlus, Map)}. */ public interface QueryRunner { /** - * Runs the given query and returns results in a time-ordered sequence - * @param query - * @param responseContext - * @return + * @deprecated use and override {@link #run(QueryPlus, Map)} instead. This method is going to be removed in Druid 0.11 */ - Sequence run(Query query, Map responseContext); + @Deprecated + default Sequence run(Query query, Map responseContext) + { + return run(QueryPlus.wrap(query), responseContext); + } + + /** + * Runs the given query and returns results in a time-ordered sequence. + */ + default Sequence run(QueryPlus queryPlus, Map responseContext) + { + return run(queryPlus.getQuery(), responseContext); + } } diff --git a/processing/src/main/java/io/druid/query/QueryRunnerHelper.java b/processing/src/main/java/io/druid/query/QueryRunnerHelper.java index 5f9a62ccdf24..ed619f2711cf 100644 --- a/processing/src/main/java/io/druid/query/QueryRunnerHelper.java +++ b/processing/src/main/java/io/druid/query/QueryRunnerHelper.java @@ -77,9 +77,9 @@ public static QueryRunner makeClosingQueryRunner(final QueryRunner ru return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - return Sequences.withBaggage(runner.run(query, responseContext), closeable); + return Sequences.withBaggage(runner.run(queryPlus, responseContext), closeable); } }; } diff --git a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java index a85a6f911286..d513e0794098 100644 --- a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -47,12 +47,12 @@ public ReferenceCountingSegmentQueryRunner( } @Override - public Sequence run(final Query query, Map responseContext) + public Sequence run(final QueryPlus queryPlus, Map responseContext) { final Closeable closeable = adapter.increment(); if (closeable != null) { try { - final Sequence baseSequence = factory.createRunner(adapter).run(query, responseContext); + final Sequence baseSequence = factory.createRunner(adapter).run(queryPlus, responseContext); return Sequences.withBaggage(baseSequence, closeable); } @@ -62,7 +62,7 @@ public Sequence run(final Query query, Map responseContext } } else { // Segment was closed before we had a chance to increment the reference count - return new ReportTimelineMissingSegmentQueryRunner(descriptor).run(query, responseContext); + return new ReportTimelineMissingSegmentQueryRunner(descriptor).run(queryPlus, responseContext); } } } diff --git a/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java index a8b6c5e2ffd6..7b60899249bb 100644 --- a/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java @@ -39,7 +39,7 @@ public ReportTimelineMissingSegmentQueryRunner(SegmentDescriptor descriptor) @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { List missingSegments = (List) responseContext.get(Result.MISSING_SEGMENTS_KEY); diff --git a/processing/src/main/java/io/druid/query/ResourceLimitExceededException.java b/processing/src/main/java/io/druid/query/ResourceLimitExceededException.java index 1ed038840ba8..06b74ccf4a57 100644 --- a/processing/src/main/java/io/druid/query/ResourceLimitExceededException.java +++ b/processing/src/main/java/io/druid/query/ResourceLimitExceededException.java @@ -19,6 +19,8 @@ package io.druid.query; +import io.druid.common.utils.StringUtils; + /** * Exception indicating that an operation failed because it exceeded some configured resource limit. * @@ -27,8 +29,8 @@ */ public class ResourceLimitExceededException extends RuntimeException { - public ResourceLimitExceededException(String message) + public ResourceLimitExceededException(String message, Object... arguments) { - super(message); + super(StringUtils.safeFormat(message, arguments)); } } diff --git a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java index b6276a107f3c..e160cbf75b59 100644 --- a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java @@ -38,9 +38,10 @@ public ResultMergeQueryRunner( } @Override - public Sequence doRun(QueryRunner baseRunner, Query query, Map context) + public Sequence doRun(QueryRunner baseRunner, QueryPlus queryPlus, Map context) { - return CombiningSequence.create(baseRunner.run(query, context), makeOrdering(query), createMergeFn(query)); + Query query = queryPlus.getQuery(); + return CombiningSequence.create(baseRunner.run(queryPlus, context), makeOrdering(query), createMergeFn(query)); } protected abstract Ordering makeOrdering(Query query); diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 57a5cde5124b..dc0a6df06e8d 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -61,10 +61,10 @@ public RetryQueryRunner( } @Override - public Sequence run(final Query query, final Map context) + public Sequence run(final QueryPlus queryPlus, final Map context) { final List> listOfSequences = Lists.newArrayList(); - listOfSequences.add(baseRunner.run(query, context)); + listOfSequences.add(baseRunner.run(queryPlus, context)); return new YieldingSequenceBase() { @@ -80,12 +80,12 @@ public Yielder toYielder( log.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), i); context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); - final Query retryQuery = query.withQuerySegmentSpec( + final QueryPlus retryQueryPlus = queryPlus.withQuerySegmentSpec( new MultipleSpecificSegmentSpec( missingSegments ) ); - Sequence retrySequence = baseRunner.run(retryQuery, context); + Sequence retrySequence = baseRunner.run(retryQueryPlus, context); listOfSequences.add(retrySequence); missingSegments = getMissingSegments(context); if (missingSegments.isEmpty()) { @@ -99,7 +99,7 @@ public Yielder toYielder( } return new MergeSequence<>( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple(listOfSequences)).toYielder( initValue, accumulator ); diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index e3dc7356c243..b2bc013e1dc8 100644 --- a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java @@ -36,13 +36,13 @@ public SubqueryQueryRunner(QueryRunner baseRunner) } @Override - public Sequence run(final Query query, Map responseContext) + public Sequence run(final QueryPlus queryPlus, Map responseContext) { - DataSource dataSource = query.getDataSource(); + DataSource dataSource = queryPlus.getQuery().getDataSource(); if (dataSource instanceof QueryDataSource) { - return run((Query) ((QueryDataSource) dataSource).getQuery(), responseContext); + return run(queryPlus.withQuery((Query) ((QueryDataSource) dataSource).getQuery()), responseContext); } else { - return baseRunner.run(query, responseContext); + return baseRunner.run(queryPlus, responseContext); } } } diff --git a/processing/src/main/java/io/druid/query/TableDataSource.java b/processing/src/main/java/io/druid/query/TableDataSource.java index e543d9b2c2f1..c558abe4c5ae 100644 --- a/processing/src/main/java/io/druid/query/TableDataSource.java +++ b/processing/src/main/java/io/druid/query/TableDataSource.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import java.util.Arrays; +import java.util.Collections; import java.util.List; @JsonTypeName("table") @@ -45,9 +45,10 @@ public String getName(){ @Override public List getNames() { - return Arrays.asList(name); + return Collections.singletonList(name); } + @Override public String toString() { return name; } @Override diff --git a/processing/src/main/java/io/druid/query/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index 0444d95d518c..df566b525027 100644 --- a/processing/src/main/java/io/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/io/druid/query/TimewarpOperator.java @@ -35,12 +35,10 @@ import java.util.Arrays; import java.util.Map; - /** * TimewarpOperator is an example post-processing operator that maps current time * to the latest period ending withing the specified data interval and truncates * the query interval to discard data that would be mapped to the future. - * */ public class TimewarpOperator implements PostProcessingOperator { @@ -49,12 +47,11 @@ public class TimewarpOperator implements PostProcessingOperator private final long originMillis; /** - * * @param dataInterval interval containing the actual data - * @param period time will be offset by a multiple of the given period - * until there is at least a full period ending within the data interval - * @param origin origin to be used to align time periods - * (e.g. to determine on what day of the week a weekly period starts) + * @param period time will be offset by a multiple of the given period + * until there is at least a full period ending within the data interval + * @param origin origin to be used to align time periods + * (e.g. to determine on what day of the week a weekly period starts) */ @JsonCreator public TimewarpOperator( @@ -69,7 +66,6 @@ public TimewarpOperator( this.periodMillis = period.toStandardDuration().getMillis(); } - @Override public QueryRunner postProcess(QueryRunner baseQueryRunner) { @@ -81,18 +77,18 @@ public QueryRunner postProcess(final QueryRunner baseRunner, final long no return new QueryRunner() { @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { final long offset = computeOffset(now); - final Interval interval = query.getIntervals().get(0); + final Interval interval = queryPlus.getQuery().getIntervals().get(0); final Interval modifiedInterval = new Interval( Math.min(interval.getStartMillis() + offset, now + offset), Math.min(interval.getEndMillis() + offset, now + offset) ); return Sequences.map( baseRunner.run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))), + queryPlus.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))), responseContext ), new Function() @@ -107,13 +103,14 @@ public T apply(T input) TimeBoundaryResultValue boundary = (TimeBoundaryResultValue) value; DateTime minTime = null; - try{ + try { minTime = boundary.getMinTime(); - } catch(IllegalArgumentException e) {} + } catch (IllegalArgumentException e) { + } final DateTime maxTime = boundary.getMaxTime(); - return (T) ((TimeBoundaryQuery) query).buildResult( + return (T) ((TimeBoundaryQuery) queryPlus.getQuery()).buildResult( new DateTime(Math.min(res.getTimestamp().getMillis() - offset, now)), minTime != null ? minTime.minus(offset) : null, maxTime != null ? new DateTime(Math.min(maxTime.getMillis() - offset, now)) : null @@ -138,6 +135,7 @@ public T apply(T input) * Map time t into the last `period` ending within `dataInterval` * * @param t the current time to be mapped into `dataInterval` + * * @return the offset between the mapped time and time t */ protected long computeOffset(final long t) @@ -145,14 +143,15 @@ protected long computeOffset(final long t) // start is the beginning of the last period ending within dataInterval long start = dataInterval.getEndMillis() - periodMillis; long startOffset = start % periodMillis - originMillis % periodMillis; - if(startOffset < 0) { + if (startOffset < 0) { startOffset += periodMillis; - }; + } + ; start -= startOffset; // tOffset is the offset time t within the last period long tOffset = t % periodMillis - originMillis % periodMillis; - if(tOffset < 0) { + if (tOffset < 0) { tOffset += periodMillis; } tOffset += start; diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index cfb337d9c94d..815838416c57 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -39,8 +39,9 @@ public UnionQueryRunner( } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { + Query query = queryPlus.getQuery(); DataSource dataSource = query.getDataSource(); if (dataSource instanceof UnionDataSource) { @@ -55,7 +56,7 @@ public Sequence run(final Query query, final Map responseC public Sequence apply(DataSource singleSource) { return baseRunner.run( - query.withDataSource(singleSource), + queryPlus.withQuery(query.withDataSource(singleSource)), responseContext ); } @@ -64,7 +65,7 @@ public Sequence apply(DataSource singleSource) ) ); } else { - return baseRunner.run(query, responseContext); + return baseRunner.run(queryPlus, responseContext); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java index 5a412fa1f555..2eb34645f90b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java @@ -41,6 +41,7 @@ public interface Aggregator extends Closeable Object get(); float getFloat(); + @Override void close(); long getLong(); diff --git a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java index 1951a6df4ae8..c45582e9e02d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -123,11 +123,13 @@ public interface BufferAggregator extends HotLoopCallee *

The default implementation inspects nothing. Classes that implement {@code BufferAggregator} are encouraged to * override this method, following the specification of {@link HotLoopCallee#inspectRuntimeShape}. */ + @Override default void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } - /* + /** * Relocates any cached objects. * If underlying ByteBuffer used for aggregation buffer relocates to a new ByteBuffer, positional caches(if any) * built on top of old ByteBuffer can not be used for further {@link BufferAggregator#aggregate(ByteBuffer, int)} diff --git a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java index daedab3f585f..ae8ee742ff71 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java @@ -68,5 +68,6 @@ public void close() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java index 36c3073e2a1c..944c940d1cf5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java @@ -30,6 +30,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -156,7 +157,7 @@ public String getName() @Override public List requiredFields() { - return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); + return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java index 7a08e5a0434a..ed12dce4e304 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java @@ -25,7 +25,7 @@ /** */ -public class DoubleMaxBufferAggregator extends DoubleBufferAggregator +public class DoubleMaxBufferAggregator extends SimpleDoubleBufferAggregator { DoubleMaxBufferAggregator(FloatColumnSelector selector) @@ -40,8 +40,18 @@ public void init(ByteBuffer buf, int position) } @Override - public void aggregate(ByteBuffer buf, int position) + public void putFirst(ByteBuffer buf, int position, double value) { - buf.putDouble(position, Math.max(buf.getDouble(position), (double) selector.get())); + if (!Double.isNaN(value)) { + buf.putDouble(position, value); + } else { + init(buf, position); + } + } + + @Override + public void aggregate(ByteBuffer buf, int position, double value) + { + buf.putDouble(position, Math.max(buf.getDouble(position), value)); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java index bec58003a329..393feab53177 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java @@ -30,6 +30,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -156,7 +157,7 @@ public String getName() @Override public List requiredFields() { - return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); + return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java index b11712818d29..427d3290b066 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java @@ -25,7 +25,7 @@ /** */ -public class DoubleMinBufferAggregator extends DoubleBufferAggregator +public class DoubleMinBufferAggregator extends SimpleDoubleBufferAggregator { DoubleMinBufferAggregator(FloatColumnSelector selector) @@ -40,8 +40,18 @@ public void init(ByteBuffer buf, int position) } @Override - public void aggregate(ByteBuffer buf, int position) + public void putFirst(ByteBuffer buf, int position, double value) { - buf.putDouble(position, Math.min(buf.getDouble(position), (double) selector.get())); + if (!Double.isNaN(value)) { + buf.putDouble(position, value); + } else { + init(buf, position); + } + } + + @Override + public void aggregate(ByteBuffer buf, int position, double value) + { + buf.putDouble(position, Math.min(buf.getDouble(position), value)); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index fa2b172fea83..266dc15b1a80 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -30,6 +30,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -156,7 +157,7 @@ public String getName() @Override public List requiredFields() { - return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); + return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java index 2c065ce49a7f..600fa646ee23 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java @@ -25,7 +25,7 @@ /** */ -public class DoubleSumBufferAggregator extends DoubleBufferAggregator +public class DoubleSumBufferAggregator extends SimpleDoubleBufferAggregator { DoubleSumBufferAggregator(FloatColumnSelector selector) @@ -40,8 +40,14 @@ public void init(ByteBuffer buf, int position) } @Override - public void aggregate(ByteBuffer buf, int position) + public void putFirst(ByteBuffer buf, int position, double value) { - buf.putDouble(position, buf.getDouble(position) + (double) selector.get()); + buf.putDouble(position, value); + } + + @Override + public void aggregate(ByteBuffer buf, int position, double value) + { + buf.putDouble(position, buf.getDouble(position) + value); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java index e777de612617..6c5a70a82e0f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java @@ -30,6 +30,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -152,7 +153,7 @@ public String getName() @Override public List requiredFields() { - return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); + return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java index 52317f4bd2f3..403d837c8f48 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java @@ -30,6 +30,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -152,7 +153,7 @@ public String getName() @Override public List requiredFields() { - return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); + return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 6c5679af332b..9c7d29713af2 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -30,6 +30,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -152,7 +153,7 @@ public String getName() @Override public List requiredFields() { - return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); + return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java index fd4d6bd51d7c..93ee651b449a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/NoopBufferAggregator.java @@ -73,5 +73,6 @@ public void close() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/SimpleDoubleBufferAggregator.java similarity index 61% rename from processing/src/main/java/io/druid/query/aggregation/DoubleBufferAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/SimpleDoubleBufferAggregator.java index 0e868674816c..33090f5463c4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/SimpleDoubleBufferAggregator.java @@ -19,34 +19,57 @@ package io.druid.query.aggregation; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; -public abstract class DoubleBufferAggregator implements BufferAggregator +public abstract class SimpleDoubleBufferAggregator implements BufferAggregator { protected final FloatColumnSelector selector; - DoubleBufferAggregator(FloatColumnSelector selector) + SimpleDoubleBufferAggregator(FloatColumnSelector selector) { this.selector = selector; } + public FloatColumnSelector getSelector() + { + return selector; + } + + /** + * Faster equivalent to + * aggregator.init(buf, position); + * aggregator.aggregate(buf, position, value); + */ + @CalledFromHotLoop + public abstract void putFirst(ByteBuffer buf, int position, double value); + + @CalledFromHotLoop + public abstract void aggregate(ByteBuffer buf, int position, double value); + + @Override + public final void aggregate(ByteBuffer buf, int position) + { + aggregate(buf, position, (double) selector.get()); + } + @Override - public Object get(ByteBuffer buf, int position) + public final Object get(ByteBuffer buf, int position) { return buf.getDouble(position); } @Override - public float getFloat(ByteBuffer buf, int position) + public final float getFloat(ByteBuffer buf, int position) { return (float) buf.getDouble(position); } @Override - public long getLong(ByteBuffer buf, int position) + public final long getLong(ByteBuffer buf, int position) { return (long) buf.getDouble(position); } diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java index ae96ff13736b..f79717eca817 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java @@ -109,7 +109,9 @@ public void reset() @Override public Object get() { - return collector; + // Workaround for non-thread-safe use of HyperLogLogCollector. + // OnheapIncrementalIndex has a penchant for calling "aggregate" and "get" simultaneously. + return HyperLogLogCollector.makeCollectorSharingStorage(collector); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java index 5ca26a7552eb..caf3476c0658 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java @@ -55,7 +55,8 @@ public void reset() @Override public Object get() { - // Workaround for OnheapIncrementalIndex's penchant for calling "aggregate" and "get" simultaneously. + // Workaround for non-thread-safe use of HyperLogLogCollector. + // OnheapIncrementalIndex has a penchant for calling "aggregate" and "get" simultaneously. return HyperLogLogCollector.makeCollectorSharingStorage(collector); } diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index b938807544ec..54e0b63b27ec 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -37,6 +37,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -193,7 +194,7 @@ public String getName() @Override public List requiredFields() { - return Arrays.asList(fieldName); + return Collections.singletonList(fieldName); } @JsonProperty diff --git a/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java index 9f22398ac0f0..46a7befb7bd5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/ArithmeticPostAggregator.java @@ -197,6 +197,7 @@ private static enum Ops { PLUS("+") { + @Override public double compute(double lhs, double rhs) { return lhs + rhs; @@ -204,6 +205,7 @@ public double compute(double lhs, double rhs) }, MINUS("-") { + @Override public double compute(double lhs, double rhs) { return lhs - rhs; @@ -211,6 +213,7 @@ public double compute(double lhs, double rhs) }, MULT("*") { + @Override public double compute(double lhs, double rhs) { return lhs * rhs; @@ -218,6 +221,7 @@ public double compute(double lhs, double rhs) }, DIV("/") { + @Override public double compute(double lhs, double rhs) { return (rhs == 0.0) ? 0 : (lhs / rhs); @@ -225,6 +229,7 @@ public double compute(double lhs, double rhs) }, QUOTIENT("quotient") { + @Override public double compute(double lhs, double rhs) { return lhs / rhs; @@ -267,6 +272,7 @@ static Set getFns() public static enum Ordering implements Comparator { // ensures the following order: numeric > NaN > Infinite numericFirst { + @Override public int compare(Double lhs, Double rhs) { if(isFinite(lhs) && !isFinite(rhs)) { return 1; diff --git a/processing/src/main/java/io/druid/query/aggregation/post/ExpressionPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/ExpressionPostAggregator.java index 149bf675aed6..a0c70107016e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/ExpressionPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/ExpressionPostAggregator.java @@ -148,6 +148,7 @@ public static enum Ordering implements Comparator { // ensures the following order: numeric > NaN > Infinite numericFirst { + @Override public int compare(Number lhs, Number rhs) { if (lhs instanceof Long && rhs instanceof Long) { diff --git a/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java index cf96a4d5afb4..0b285c3e8c02 100644 --- a/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/post/JavaScriptPostAggregator.java @@ -68,6 +68,7 @@ private static Function compile(String function) return new Function() { + @Override public double apply(Object[] args) { // ideally we need a close() function to discard the context once it is not used anymore diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java index e987fe2c4fb6..d750c97cc556 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java @@ -34,7 +34,6 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -57,7 +56,7 @@ public DataSourceMetadataQuery( super( dataSource, (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(MY_Y2K_INTERVAL)) - : querySegmentSpec, + : querySegmentSpec, false, context ); @@ -102,10 +101,12 @@ public Query> withDataSource(DataSource da public Iterable> buildResult(DateTime timestamp, DateTime maxIngestedEventTime) { - return Arrays.asList(new Result<>(timestamp, new DataSourceMetadataResultValue(maxIngestedEventTime))); + return Collections.singletonList(new Result<>(timestamp, new DataSourceMetadataResultValue(maxIngestedEventTime))); } - public Iterable> mergeResults(List> results) + public Iterable> mergeResults( + List> results + ) { if (results == null || results.isEmpty()) { return Lists.newArrayList(); @@ -126,10 +127,10 @@ public Iterable> mergeResults(List> query = input.getQuery(); + if (!(query instanceof DataSourceMetadataQuery)) { + throw new ISE("Got a [%s] which isn't a %s", query.getClass().getCanonicalName(), DataSourceMetadataQuery.class); } - final DataSourceMetadataQuery legacyQuery = (DataSourceMetadataQuery) input; + final DataSourceMetadataQuery legacyQuery = (DataSourceMetadataQuery) query; return new BaseSequence<>( new BaseSequence.IteratorMaker, Iterator>>() diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index 1c5b2eb00ea7..03b52599cc31 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -33,6 +33,7 @@ import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -93,15 +94,15 @@ public QueryRunner> mergeResults( @Override protected Sequence> doRun( QueryRunner> baseRunner, - Query> input, + QueryPlus> input, Map context ) { - DataSourceMetadataQuery query = (DataSourceMetadataQuery) input; + DataSourceMetadataQuery query = (DataSourceMetadataQuery) input.getQuery(); return Sequences.simple( query.mergeResults( Sequences.toList( - baseRunner.run(query, context), + baseRunner.run(input, context), Lists.>newArrayList() ) ) diff --git a/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java b/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java index 52c489cce6eb..fa19679fef6d 100644 --- a/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java +++ b/processing/src/main/java/io/druid/query/dimension/ForwardingFilteredDimensionSelector.java @@ -31,7 +31,7 @@ import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.BooleanValueMatcher; -import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; import javax.annotation.Nullable; import java.util.BitSet; @@ -40,14 +40,18 @@ final class ForwardingFilteredDimensionSelector implements DimensionSelector, Id { private final DimensionSelector selector; private final IdLookup baseIdLookup; - private final Int2IntMap forwardMapping; + private final Int2IntOpenHashMap forwardMapping; private final int[] reverseMapping; /** * @param selector must return true from {@link DimensionSelector#nameLookupPossibleInAdvance()} - * @param forwardMapping must have {@link Int2IntMap#defaultReturnValue(int)} configured to -1. + * @param forwardMapping must have {@link Int2IntOpenHashMap#defaultReturnValue(int)} configured to -1. */ - ForwardingFilteredDimensionSelector(DimensionSelector selector, Int2IntMap forwardMapping, int[] reverseMapping) + ForwardingFilteredDimensionSelector( + DimensionSelector selector, + Int2IntOpenHashMap forwardMapping, + int[] reverseMapping + ) { this.selector = Preconditions.checkNotNull(selector); if (!selector.nameLookupPossibleInAdvance()) { @@ -106,6 +110,12 @@ public boolean matches() // null should match empty rows in multi-value columns return nullRow && value == null; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } else { return BooleanValueMatcher.of(false); @@ -141,6 +151,12 @@ public boolean matches() // null should match empty rows in multi-value columns return nullRow && matchNull; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } @@ -179,6 +195,5 @@ public int lookupId(String name) public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("selector", selector); - inspector.visit("forwardMapping", forwardMapping); } } diff --git a/processing/src/main/java/io/druid/query/dimension/ListFilteredDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ListFilteredDimensionSpec.java index c1234f6ebcff..c7ab3873b578 100644 --- a/processing/src/main/java/io/druid/query/dimension/ListFilteredDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ListFilteredDimensionSpec.java @@ -28,7 +28,6 @@ import io.druid.query.filter.DimFilterUtils; import io.druid.segment.DimensionSelector; import io.druid.segment.IdLookup; -import it.unimi.dsi.fastutil.ints.Int2IntMap; import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; import javax.annotation.Nullable; @@ -93,7 +92,7 @@ private DimensionSelector filterWhiteList(DimensionSelector selector) } final int maxPossibleFilteredCardinality = values.size(); int count = 0; - final Int2IntMap forwardMapping = new Int2IntOpenHashMap(maxPossibleFilteredCardinality); + final Int2IntOpenHashMap forwardMapping = new Int2IntOpenHashMap(maxPossibleFilteredCardinality); forwardMapping.defaultReturnValue(-1); final int[] reverseMapping = new int[maxPossibleFilteredCardinality]; IdLookup idLookup = selector.idLookup(); @@ -134,7 +133,7 @@ public boolean apply(@Nullable String input) } final int maxPossibleFilteredCardinality = selectorCardinality; int count = 0; - final Int2IntMap forwardMapping = new Int2IntOpenHashMap(maxPossibleFilteredCardinality); + final Int2IntOpenHashMap forwardMapping = new Int2IntOpenHashMap(maxPossibleFilteredCardinality); forwardMapping.defaultReturnValue(-1); final int[] reverseMapping = new int[maxPossibleFilteredCardinality]; for (int i = 0; i < selectorCardinality; i++) { diff --git a/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java b/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java index 44b9fc347471..a99eda5c521e 100644 --- a/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java +++ b/processing/src/main/java/io/druid/query/dimension/PredicateFilteredDimensionSelector.java @@ -79,6 +79,13 @@ public boolean matches() // null should match empty rows in multi-value columns return nullRow && value == null; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // PredicateFilteredDimensionSelector.this inspects selector and predicate as well. + inspector.visit("selector", PredicateFilteredDimensionSelector.this); + } }; } @@ -106,6 +113,14 @@ public boolean matches() // null should match empty rows in multi-value columns return nullRow && matchNull; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // PredicateFilteredDimensionSelector.this inspects selector and predicate as well. + inspector.visit("selector", PredicateFilteredDimensionSelector.this); + inspector.visit("matcherPredicate", matcherPredicate); + } }; } diff --git a/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java index 3cfd7361488a..4ebd407083f4 100644 --- a/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java @@ -83,7 +83,7 @@ public boolean apply(@Nullable String input) } int count = 0; - final Int2IntMap forwardMapping = new Int2IntOpenHashMap(); + final Int2IntOpenHashMap forwardMapping = new Int2IntOpenHashMap(); forwardMapping.defaultReturnValue(-1); for (int i = 0; i < selectorCardinality; i++) { if (compiledRegex.matcher(Strings.nullToEmpty(selector.lookupName(i))).matches()) { diff --git a/processing/src/main/java/io/druid/query/extraction/CascadeExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/CascadeExtractionFn.java index b3d09805ca11..22e51b726c36 100644 --- a/processing/src/main/java/io/druid/query/extraction/CascadeExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/CascadeExtractionFn.java @@ -34,31 +34,37 @@ public class CascadeExtractionFn implements ExtractionFn private final ChainedExtractionFn DEFAULT_CHAINED_EXTRACTION_FN = new ChainedExtractionFn( new ExtractionFn() { + @Override public byte[] getCacheKey() { return new byte[0]; } + @Override public String apply(Object value) { return null; } + @Override public String apply(String value) { return null; } + @Override public String apply(long value) { return null; } + @Override public boolean preservesOrdering() { return false; } + @Override public ExtractionType getExtractionType() { return ExtractionType.MANY_TO_ONE; @@ -171,7 +177,7 @@ public String toString() "extractionFns=[" + chainedExtractionFn.toString() + "]}"; } - private class ChainedExtractionFn + private static class ChainedExtractionFn { private final ExtractionFn fn; private final ChainedExtractionFn child; @@ -219,6 +225,7 @@ public ExtractionType getExtractionType() } } + @Override public boolean equals(Object o) { if (this == o) { @@ -240,6 +247,7 @@ public boolean equals(Object o) return true; } + @Override public int hashCode() { int result = fn.hashCode(); @@ -249,6 +257,7 @@ public int hashCode() return result; } + @Override public String toString() { return (child != null) diff --git a/processing/src/main/java/io/druid/query/extraction/JavaScriptExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/JavaScriptExtractionFn.java index eaa7ce7f3eb3..bfb7987b0d0d 100644 --- a/processing/src/main/java/io/druid/query/extraction/JavaScriptExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/JavaScriptExtractionFn.java @@ -50,6 +50,7 @@ private static Function compile(String function) return new Function() { + @Override public String apply(Object input) { // ideally we need a close() function to discard the context once it is not used anymore diff --git a/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java index 2e16149c1038..b209f1b65345 100644 --- a/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/filter/FloatValueMatcherColumnSelectorStrategy.java @@ -19,6 +19,7 @@ package io.druid.query.filter; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionHandlerUtils; import io.druid.segment.FloatColumnSelector; import io.druid.segment.filter.BooleanValueMatcher; @@ -41,6 +42,12 @@ public boolean matches() { return Float.floatToIntBits(selector.get()) == matchValIntBits; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } @@ -57,6 +64,13 @@ public boolean matches() { return predicate.applyFloat(selector.get()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("predicate", predicate); + } }; } diff --git a/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java index a0a2f59bf0dd..3995d9e0ae76 100644 --- a/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/filter/LongValueMatcherColumnSelectorStrategy.java @@ -19,6 +19,7 @@ package io.druid.query.filter; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionHandlerUtils; import io.druid.segment.LongColumnSelector; import io.druid.segment.filter.BooleanValueMatcher; @@ -40,6 +41,12 @@ public boolean matches() { return selector.get() == matchValLong; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } @@ -56,6 +63,13 @@ public boolean matches() { return predicate.applyLong(selector.get()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("predicate", predicate); + } }; } diff --git a/processing/src/main/java/io/druid/query/filter/NotDimFilter.java b/processing/src/main/java/io/druid/query/filter/NotDimFilter.java index cf4de057c0cd..978a685d6775 100644 --- a/processing/src/main/java/io/druid/query/filter/NotDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/NotDimFilter.java @@ -37,6 +37,7 @@ public class NotDimFilter implements DimFilter { private static final Function NEGATE = new Function() { + @Override public DimFilter apply(DimFilter filter) { return Druids.newNotDimFilterBuilder().field(filter).build(); } }; diff --git a/processing/src/main/java/io/druid/query/filter/ValueMatcher.java b/processing/src/main/java/io/druid/query/filter/ValueMatcher.java index b701f3ade385..cad255474d8b 100644 --- a/processing/src/main/java/io/druid/query/filter/ValueMatcher.java +++ b/processing/src/main/java/io/druid/query/filter/ValueMatcher.java @@ -19,9 +19,13 @@ package io.druid.query.filter; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; + /** */ -public interface ValueMatcher +public interface ValueMatcher extends HotLoopCallee { - public boolean matches(); + @CalledFromHotLoop + boolean matches(); } diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java index 5d8ee7321fc3..808269e949a1 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java @@ -43,19 +43,19 @@ public void query(GroupByQuery query) @Override public void numDimensions(GroupByQuery query) { - builder.setDimension("numDimensions", String.valueOf(query.getDimensions().size())); + setDimension("numDimensions", String.valueOf(query.getDimensions().size())); } @Override public void numMetrics(GroupByQuery query) { - builder.setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); } @Override public void numComplexMetrics(GroupByQuery query) { int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); - builder.setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); } } diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java index f70c9c6c9872..dd9b3b056c40 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java @@ -22,8 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; +import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; +@LazySingleton public class DefaultGroupByQueryMetricsFactory implements GroupByQueryMetricsFactory { private static final GroupByQueryMetricsFactory INSTANCE = @@ -43,7 +46,7 @@ public static GroupByQueryMetricsFactory instance() private final ObjectMapper jsonMapper; @Inject - public DefaultGroupByQueryMetricsFactory(ObjectMapper jsonMapper) + public DefaultGroupByQueryMetricsFactory(@Json ObjectMapper jsonMapper) { this.jsonMapper = jsonMapper; } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 553e8a2a95c0..893bdd13d26f 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -25,6 +25,7 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; @@ -53,17 +54,22 @@ import io.druid.query.groupby.orderby.LimitSpec; import io.druid.query.groupby.orderby.NoopLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; +import io.druid.query.groupby.strategy.GroupByStrategyV2; +import io.druid.query.ordering.StringComparator; +import io.druid.query.ordering.StringComparators; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import io.druid.segment.VirtualColumn; import io.druid.segment.VirtualColumns; import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -96,6 +102,8 @@ public static Builder builder() private final List aggregatorSpecs; private final List postAggregatorSpecs; + private final Function, Sequence> limitFn; + private final boolean applyLimitPushDown; private final Function, Sequence> postProcessingFn; @JsonCreator @@ -189,6 +197,45 @@ private GroupByQuery( verifyOutputNames(this.dimensions, this.aggregatorSpecs, this.postAggregatorSpecs); this.postProcessingFn = postProcessingFn != null ? postProcessingFn : makePostProcessingFn(); + + // Check if limit push down configuration is valid and check if limit push down will be applied + this.applyLimitPushDown = determineApplyLimitPushDown(); + + // On an inner query, we may sometimes get a LimitSpec so that row orderings can be determined for limit push down + // However, it's not necessary to build the real limitFn from it at this stage. + Function, Sequence> postProcFn; + if (getContextBoolean(GroupByStrategyV2.CTX_KEY_OUTERMOST, true)) { + postProcFn = this.limitSpec.build(this.dimensions, this.aggregatorSpecs, this.postAggregatorSpecs); + } else { + postProcFn = NoopLimitSpec.INSTANCE.build(this.dimensions, this.aggregatorSpecs, this.postAggregatorSpecs); + } + + if (havingSpec != null) { + postProcFn = Functions.compose( + postProcFn, + new Function, Sequence>() + { + @Override + public Sequence apply(Sequence input) + { + GroupByQuery.this.havingSpec.setRowSignature(GroupByQueryHelper.rowSignatureFor(GroupByQuery.this)); + return Sequences.filter( + input, + new Predicate() + { + @Override + public boolean apply(Row input) + { + return GroupByQuery.this.havingSpec.eval(input); + } + } + ); + } + } + ); + } + + limitFn = postProcFn; } @JsonProperty @@ -263,6 +310,12 @@ public boolean getContextSortByDimsFirst() return getContextBoolean(CTX_KEY_SORT_BY_DIMS_FIRST, false); } + @JsonIgnore + public boolean isApplyLimitPushDown() + { + return applyLimitPushDown; + } + @Override public Ordering getResultOrdering() { @@ -280,10 +333,177 @@ public Ordering getResultOrdering() ); } - public Ordering getRowOrdering(final boolean granular) + private boolean validateAndGetForceLimitPushDown() + { + final boolean forcePushDown = getContextBoolean(GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, false); + if (forcePushDown) { + if (!(limitSpec instanceof DefaultLimitSpec)) { + throw new IAE("When forcing limit push down, a limit spec must be provided."); + } + + if (((DefaultLimitSpec) limitSpec).getLimit() == Integer.MAX_VALUE) { + throw new IAE("When forcing limit push down, the provided limit spec must have a limit."); + } + + for (OrderByColumnSpec orderBySpec : ((DefaultLimitSpec) limitSpec).getColumns()) { + if (OrderByColumnSpec.getPostAggIndexForOrderBy(orderBySpec, postAggregatorSpecs) > -1) { + throw new UnsupportedOperationException("Limit push down when sorting by a post aggregator is not supported."); + } + } + } + return forcePushDown; + } + + public boolean determineApplyLimitPushDown() + { + final boolean forceLimitPushDown = validateAndGetForceLimitPushDown(); + + if (limitSpec instanceof DefaultLimitSpec) { + DefaultLimitSpec defaultLimitSpec = (DefaultLimitSpec) limitSpec; + + // If only applying an orderby without a limit, don't try to push down + if (defaultLimitSpec.getLimit() == Integer.MAX_VALUE) { + return false; + } + + if (forceLimitPushDown) { + return true; + } + + // If the sorting order only uses columns in the grouping key, we can always push the limit down + // to the buffer grouper without affecting result accuracy + boolean sortHasNonGroupingFields = DefaultLimitSpec.sortingOrderHasNonGroupingFields( + (DefaultLimitSpec) limitSpec, + getDimensions() + ); + + return !sortHasNonGroupingFields; + } + + return false; + } + + /** + * When limit push down is applied, the partial results would be sorted by the ordering specified by the + * limit/order spec (unlike non-push down case where the results always use the default natural ascending order), + * so when merging these partial result streams, the merge needs to use the same ordering to get correct results. + */ + private Ordering getRowOrderingForPushDown( + final boolean granular, + final DefaultLimitSpec limitSpec + ) { final boolean sortByDimsFirst = getContextSortByDimsFirst(); + final List orderedFieldNames = new ArrayList<>(); + final Set dimsInOrderBy = new HashSet<>(); + final List needsReverseList = new ArrayList<>(); + final List isNumericField = new ArrayList<>(); + final List comparators = new ArrayList<>(); + + for (OrderByColumnSpec orderSpec : limitSpec.getColumns()) { + boolean needsReverse = orderSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING; + int dimIndex = OrderByColumnSpec.getDimIndexForOrderBy(orderSpec, dimensions); + if (dimIndex >= 0) { + DimensionSpec dim = dimensions.get(dimIndex); + orderedFieldNames.add(dim.getOutputName()); + dimsInOrderBy.add(dimIndex); + needsReverseList.add(needsReverse); + final ValueType type = dimensions.get(dimIndex).getOutputType(); + isNumericField.add(type == ValueType.LONG || type == ValueType.FLOAT); + comparators.add(orderSpec.getDimensionComparator()); + } + } + + for (int i = 0; i < dimensions.size(); i++) { + if (!dimsInOrderBy.contains(i)) { + orderedFieldNames.add(dimensions.get(i).getOutputName()); + needsReverseList.add(false); + final ValueType type = dimensions.get(i).getOutputType(); + isNumericField.add(type == ValueType.LONG || type == ValueType.FLOAT); + comparators.add(StringComparators.LEXICOGRAPHIC); + } + } + + final Comparator timeComparator = getTimeComparator(granular); + + if (timeComparator == null) { + return Ordering.from( + new Comparator() + { + @Override + public int compare(Row lhs, Row rhs) + { + return compareDimsForLimitPushDown( + orderedFieldNames, + needsReverseList, + isNumericField, + comparators, + lhs, + rhs + ); + } + } + ); + } else if (sortByDimsFirst) { + return Ordering.from( + new Comparator() + { + @Override + public int compare(Row lhs, Row rhs) + { + final int cmp = compareDimsForLimitPushDown( + orderedFieldNames, + needsReverseList, + isNumericField, + comparators, + lhs, + rhs + ); + if (cmp != 0) { + return cmp; + } + + return timeComparator.compare(lhs, rhs); + } + } + ); + } else { + return Ordering.from( + new Comparator() + { + @Override + public int compare(Row lhs, Row rhs) + { + final int timeCompare = timeComparator.compare(lhs, rhs); + + if (timeCompare != 0) { + return timeCompare; + } + + return compareDimsForLimitPushDown( + orderedFieldNames, + needsReverseList, + isNumericField, + comparators, + lhs, + rhs + ); + } + } + ); + } + } + + public Ordering getRowOrdering(final boolean granular) + { + if (applyLimitPushDown) { + if (!DefaultLimitSpec.sortingOrderHasNonGroupingFields((DefaultLimitSpec) limitSpec, dimensions)) { + return getRowOrderingForPushDown(granular, (DefaultLimitSpec) limitSpec); + } + } + + final boolean sortByDimsFirst = getContextSortByDimsFirst(); final Comparator timeComparator = getTimeComparator(granular); if (timeComparator == null) { @@ -331,10 +551,23 @@ private Comparator getTimeComparator(boolean granular) private static int compareDims(List dimensions, Row lhs, Row rhs) { for (DimensionSpec dimension : dimensions) { - final int dimCompare = NATURAL_NULLS_FIRST.compare( - lhs.getRaw(dimension.getOutputName()), - rhs.getRaw(dimension.getOutputName()) - ); + final int dimCompare; + if (dimension.getOutputType() == ValueType.LONG) { + dimCompare = Long.compare( + ((Number) lhs.getRaw(dimension.getOutputName())).longValue(), + ((Number) rhs.getRaw(dimension.getOutputName())).longValue() + ); + } else if (dimension.getOutputType() == ValueType.FLOAT) { + dimCompare = Double.compare( + ((Number) lhs.getRaw(dimension.getOutputName())).doubleValue(), + ((Number) rhs.getRaw(dimension.getOutputName())).doubleValue() + ); + } else { + dimCompare = NATURAL_NULLS_FIRST.compare( + lhs.getRaw(dimension.getOutputName()), + rhs.getRaw(dimension.getOutputName()) + ); + } if (dimCompare != 0) { return dimCompare; } @@ -343,6 +576,51 @@ private static int compareDims(List dimensions, Row lhs, Row rhs) return 0; } + private static int compareDimsForLimitPushDown( + final List fields, + final List needsReverseList, + final List isNumericField, + final List comparators, + Row lhs, + Row rhs + ) + { + for (int i = 0; i < fields.size(); i++) { + final String fieldName = fields.get(i); + final StringComparator comparator = comparators.get(i); + + final int dimCompare; + + Object lhsObj; + Object rhsObj; + if (needsReverseList.get(i)) { + lhsObj = rhs.getRaw(fieldName); + rhsObj = lhs.getRaw(fieldName); + } else { + lhsObj = lhs.getRaw(fieldName); + rhsObj = rhs.getRaw(fieldName); + } + + if (isNumericField.get(i)) { + if (comparator == StringComparators.NUMERIC) { + dimCompare = NATURAL_NULLS_FIRST.compare( + rhs.getRaw(fieldName), + lhs.getRaw(fieldName) + ); + } else { + dimCompare = comparator.compare(String.valueOf(lhsObj), String.valueOf(rhsObj)); + } + } else { + dimCompare = comparator.compare((String) lhsObj, (String) rhsObj); + } + + if (dimCompare != 0) { + return dimCompare; + } + } + return 0; + } + /** * Apply the havingSpec and limitSpec. Because havingSpecs are not thread safe, and because they are applied during * accumulation of the returned sequence, callers must take care to avoid accumulating two different Sequences diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java index 70090210ae6f..6f55b9a2abc7 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -27,6 +27,7 @@ public class GroupByQueryConfig { public static final String CTX_KEY_STRATEGY = "groupByStrategy"; + public static final String CTX_KEY_FORCE_LIMIT_PUSH_DOWN = "forceLimitPushDown"; private static final String CTX_KEY_IS_SINGLE_THREADED = "groupByIsSingleThreaded"; private static final String CTX_KEY_MAX_INTERMEDIATE_ROWS = "maxIntermediateRows"; private static final String CTX_KEY_MAX_RESULTS = "maxResults"; @@ -66,6 +67,12 @@ public class GroupByQueryConfig // Max on-disk temporary storage, per-query; when exceeded, the query fails private long maxOnDiskStorage = 0L; + @JsonProperty + private boolean forcePushDownLimit = false; + + @JsonProperty + private Class queryMetricsFactory; + public String getDefaultStrategy() { return defaultStrategy; @@ -126,6 +133,21 @@ public long getMaxOnDiskStorage() return maxOnDiskStorage; } + public boolean isForcePushDownLimit() + { + return forcePushDownLimit; + } + + public Class getQueryMetricsFactory() + { + return queryMetricsFactory != null ? queryMetricsFactory : DefaultGroupByQueryMetricsFactory.class; + } + + public void setQueryMetricsFactory(Class queryMetricsFactory) + { + this.queryMetricsFactory = queryMetricsFactory; + } + public GroupByQueryConfig withOverrides(final GroupByQuery query) { final GroupByQueryConfig newConfig = new GroupByQueryConfig(); @@ -159,6 +181,7 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) ((Number) query.getContextValue(CTX_KEY_MAX_MERGING_DICTIONARY_SIZE, getMaxMergingDictionarySize())).longValue(), getMaxMergingDictionarySize() ); + newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit()); return newConfig; } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index fb518aa109ee..621298a34c99 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -447,6 +447,7 @@ public void remove() throw new UnsupportedOperationException(); } + @Override public void close() { // cleanup diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java index da76ddc79e2a..9061e411b6a1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java @@ -19,6 +19,17 @@ package io.druid.query.groupby; +/** + * Implementations could be injected using + * + * PolyBind + * .optionBinder(binder, Key.get(GroupByQueryMetricsFactory.class)) + * .addBinding("myCustomGroupByQueryMetricsFactory") + * .to(MyCustomGroupByQueryMetricsFactory.class); + * + * And then setting property: + * druid.query.groupBy.queryMetricsFactory=myCustomGroupByQueryMetricsFactory + */ public interface GroupByQueryMetricsFactory { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index ddf9c4277c3f..19d8c541d7da 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -41,9 +41,9 @@ import io.druid.query.CacheStrategy; import io.druid.query.DataSource; import io.druid.query.IntervalChunkingQueryRunnerDecorator; -import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; @@ -113,13 +113,13 @@ public QueryRunner mergeResults(final QueryRunner runner) return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - if (QueryContexts.isBySegment(query)) { - return runner.run(query, responseContext); + if (QueryContexts.isBySegment(queryPlus.getQuery())) { + return runner.run(queryPlus, responseContext); } - final GroupByQuery groupByQuery = (GroupByQuery) query; + final GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); if (strategySelector.strategize(groupByQuery).doMergeResults(groupByQuery)) { return initAndMergeGroupByResults( groupByQuery, @@ -127,7 +127,7 @@ public Sequence run(Query query, Map responseContext) responseContext ); } - return runner.run(query, responseContext); + return runner.run(queryPlus, responseContext); } }; } @@ -327,9 +327,9 @@ public QueryRunner preMergeQueryDecoration(final QueryRunner runner) new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - GroupByQuery groupByQuery = (GroupByQuery) query; + GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); if (groupByQuery.getDimFilter() != null) { groupByQuery = groupByQuery.withDimFilter(groupByQuery.getDimFilter().optimize()); } @@ -365,7 +365,7 @@ public String apply(DimensionSpec input) GroupByQueryQueryToolChest.this ) .run( - delegateGroupByQuery.withDimensionSpecs(dimensionSpecs), + queryPlus.withQuery(delegateGroupByQuery.withDimensionSpecs(dimensionSpecs)), responseContext ); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java index 5f677e0cf7c6..c35372476867 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -26,6 +26,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -68,12 +69,13 @@ public QueryRunner mergeRunners(final ExecutorService exec, final Iterable< return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - return strategySelector.strategize((GroupByQuery) query).mergeRunners(queryExecutor, queryRunners).run( - query, - responseContext + QueryRunner rowQueryRunner = strategySelector.strategize((GroupByQuery) queryPlus.getQuery()).mergeRunners( + queryExecutor, + queryRunners ); + return rowQueryRunner.run(queryPlus, responseContext); } }; } @@ -96,13 +98,14 @@ public GroupByQueryRunner(Segment segment, final GroupByStrategySelector strateg } @Override - public Sequence run(Query input, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - if (!(input instanceof GroupByQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); + Query query = queryPlus.getQuery(); + if (!(query instanceof GroupByQuery)) { + throw new ISE("Got a [%s] which isn't a %s", query.getClass(), GroupByQuery.class); } - return strategySelector.strategize((GroupByQuery) input).process((GroupByQuery) input, adapter); + return strategySelector.strategize((GroupByQuery) query).process((GroupByQuery) query, adapter); } } } diff --git a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java index d8d15070a8bd..3fcb2434ae90 100644 --- a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java @@ -34,6 +34,7 @@ import io.druid.segment.IdLookup; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.SingleValueDimensionSelector; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; @@ -105,7 +106,7 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi throw new UnsupportedOperationException("time dimension must provide an extraction function"); } - return new DimensionSelector() + return new SingleValueDimensionSelector() { @Override public IndexedInts getRow() @@ -113,6 +114,12 @@ public IndexedInts getRow() return ZeroIndexedInts.instance(); } + @Override + public int getRowValue() + { + return 0; + } + @Override public ValueMatcher makeValueMatcher(final String value) { @@ -124,6 +131,13 @@ public boolean matches() String rowValue = extractionFn.apply(row.get().getTimestampFromEpoch()); return Objects.equals(rowValue, value); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("extractionFn", extractionFn); + } }; } @@ -138,6 +152,14 @@ public boolean matches() String rowValue = extractionFn.apply(row.get().getTimestampFromEpoch()); return predicate.apply(rowValue); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("extractionFn", extractionFn); + inspector.visit("predicate", predicate); + } }; } @@ -204,6 +226,12 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + } }; } else { return new ValueMatcher() @@ -223,6 +251,13 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("extractionFn", extractionFn); + } }; } } @@ -249,6 +284,13 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("predicate", predicate); + } }; } else { return new ValueMatcher() @@ -268,6 +310,13 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("row", row); + inspector.visit("predicate", predicate); + } }; } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java new file mode 100644 index 000000000000..a0b5e8d10cef --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/AbstractBufferGrouper.java @@ -0,0 +1,214 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Supplier; +import com.google.common.primitives.Ints; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; + +import java.nio.ByteBuffer; + +public abstract class AbstractBufferGrouper implements Grouper +{ + private static final AggregateResult DICTIONARY_FULL = AggregateResult.failure( + "Not enough dictionary space to execute this query. Try increasing " + + "druid.query.groupBy.maxMergingDictionarySize or enable disk spilling by setting " + + "druid.query.groupBy.maxOnDiskStorage to a positive number." + ); + private static final AggregateResult HASHTABLE_FULL = AggregateResult.failure( + "Not enough aggregation table space to execute this query. Try increasing " + + "druid.processing.buffer.sizeBytes or enable disk spilling by setting " + + "druid.query.groupBy.maxOnDiskStorage to a positive number." + ); + + protected static final int HASH_SIZE = Ints.BYTES; + protected static final Logger log = new Logger(AbstractBufferGrouper.class); + + protected final Supplier bufferSupplier; + protected final KeySerde keySerde; + protected final int keySize; + protected final BufferAggregator[] aggregators; + protected final int[] aggregatorOffsets; + protected final int bufferGrouperMaxSize; // Integer.MAX_VALUE in production, only used for unit tests + + // The load factor and bucket configurations are not final, to allow subclasses to set their own values + protected float maxLoadFactor; + protected int initialBuckets; + protected int bucketSize; + + // The hashTable and its buffer are not final, these are set during init() for buffer management purposes + // See PR 3863 for details: https://github.com/druid-io/druid/pull/3863 + protected ByteBufferHashTable hashTable; + protected ByteBuffer hashTableBuffer; // buffer for the entire hash table (total space, not individual growth) + + public AbstractBufferGrouper( + final Supplier bufferSupplier, + final KeySerde keySerde, + final AggregatorFactory[] aggregatorFactories, + final int bufferGrouperMaxSize + ) + { + this.bufferSupplier = bufferSupplier; + this.keySerde = keySerde; + this.keySize = keySerde.keySize(); + this.aggregators = new BufferAggregator[aggregatorFactories.length]; + this.aggregatorOffsets = new int[aggregatorFactories.length]; + this.bufferGrouperMaxSize = bufferGrouperMaxSize; + } + + /** + * Called when a new bucket is used for an entry in the hash table. An implementing BufferGrouper class + * can use this to update its own state, e.g. tracking bucket offsets in a structure outside of the hash table. + * + * @param bucketOffset offset of the new bucket, within the buffer returned by hashTable.getTableBuffer() + */ + public abstract void newBucketHook(int bucketOffset); + + /** + * Called to check if it's possible to skip aggregation for a row. + * + * @param bucketWasUsed Was the row a new entry in the hash table? + * @param bucketOffset Offset of the bucket containing this row's entry in the hash table, + * within the buffer returned by hashTable.getTableBuffer() + * @return true if aggregation can be skipped, false otherwise. + */ + public abstract boolean canSkipAggregate(boolean bucketWasUsed, int bucketOffset); + + /** + * Called after a row is aggregated. An implementing BufferGrouper class can use this to update + * its own state, e.g. reading the new aggregated values for the row's key and acting on that information. + * + * @param bucketOffset Offset of the bucket containing the row that was aggregated, + * within the buffer returned by hashTable.getTableBuffer() + */ + public abstract void afterAggregateHook(int bucketOffset); + + // how many times the hash table's buffer has filled/readjusted (through adjustTableWhenFull()) + public int getGrowthCount() + { + return hashTable.getGrowthCount(); + } + + // Number of elements in the table right now + public int getSize() + { + return hashTable.getSize(); + } + + // Current number of available/used buckets in the table + public int getBuckets() + { + return hashTable.getMaxBuckets(); + } + + // Maximum number of elements in the table before it must be resized + public int getMaxSize() + { + return hashTable.getRegrowthThreshold(); + } + + @Override + public AggregateResult aggregate(KeyType key, int keyHash) + { + final ByteBuffer keyBuffer = keySerde.toByteBuffer(key); + if (keyBuffer == null) { + // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will + // be correct. + return DICTIONARY_FULL; + } + + if (keyBuffer.remaining() != keySize) { + throw new IAE( + "keySerde.toByteBuffer(key).remaining[%s] != keySerde.keySize[%s], buffer was the wrong size?!", + keyBuffer.remaining(), + keySize + ); + } + + // find and try to expand if table is full and find again + int bucket = hashTable.findBucketWithAutoGrowth(keyBuffer, keyHash); + if (bucket < 0) { + // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will + // be correct. + return HASHTABLE_FULL; + } + + final int bucketStartOffset = hashTable.getOffsetForBucket(bucket); + final boolean bucketWasUsed = hashTable.isBucketUsed(bucket); + final ByteBuffer tableBuffer = hashTable.getTableBuffer(); + + // Set up key and initialize the aggs if this is a new bucket. + if (!bucketWasUsed) { + hashTable.initializeNewBucketKey(bucket, keyBuffer, keyHash); + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].init(tableBuffer, bucketStartOffset + aggregatorOffsets[i]); + } + + newBucketHook(bucketStartOffset); + } + + if (canSkipAggregate(bucketWasUsed, bucketStartOffset)) { + return AggregateResult.ok(); + } + + // Aggregate the current row. + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].aggregate(tableBuffer, bucketStartOffset + aggregatorOffsets[i]); + } + + afterAggregateHook(bucketStartOffset); + + return AggregateResult.ok(); + } + + @Override + public AggregateResult aggregate(final KeyType key) + { + return aggregate(key, Groupers.hash(key)); + } + + @Override + public void close() + { + for (BufferAggregator aggregator : aggregators) { + try { + aggregator.close(); + } + catch (Exception e) { + log.warn(e, "Could not close aggregator, skipping.", aggregator); + } + } + } + + protected Entry bucketEntryForOffset(final int bucketOffset) + { + final ByteBuffer tableBuffer = hashTable.getTableBuffer(); + final KeyType key = keySerde.fromByteBuffer(tableBuffer, bucketOffset + HASH_SIZE); + final Object[] values = new Object[aggregators.length]; + for (int i = 0; i < aggregators.length; i++) { + values[i] = aggregators[i].get(tableBuffer, bucketOffset + aggregatorOffsets[i]); + } + + return new Entry<>(key, values); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java index 9c4a25da2c8e..cb738ed4d0cd 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java @@ -20,12 +20,11 @@ package io.druid.query.groupby.epinephelinae; import com.google.common.base.Supplier; +import com.google.common.collect.Iterators; import com.google.common.primitives.Ints; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.ColumnSelectorFactory; import java.nio.ByteBuffer; @@ -34,77 +33,24 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.NoSuchElementException; -/** - * Grouper based around a hash table and companion array in a single ByteBuffer. Not thread-safe. - * - * The buffer has two parts: a table arena (offset 0 to tableArenaSize) and an array containing pointers objects in - * the table (tableArenaSize until the end of the buffer). - * - * The table uses open addressing with linear probing on collisions. Each bucket contains the key hash (with the high - * bit set to signify the bucket is used), the serialized key (which are a fixed size) and scratch space for - * BufferAggregators (which is also fixed size). The actual table is represented by "tableBuffer", which points to the - * same memory as positions "tableStart" through "tableStart + buckets * bucketSize" of "buffer". Everything else in - * the table arena is potentially junk. - * - * The array of pointers starts out ordered by insertion order, but might be sorted on calls to - * {@link #iterator(boolean)}. This sorting is done in-place to avoid materializing the full array of pointers. The - * first "size" pointers in the array of pointers are valid; everything else is potentially junk. - * - * The table is periodically grown to accommodate more keys. Even though starting small is not necessary to control - * memory use (we already have the entire buffer allocated) or iteration speed (iteration is fast due to the array - * of pointers) it still helps significantly on initialization times. Otherwise, we'd need to clear the used bits of - * each bucket in the entire buffer, which is a lot of writes if the buckets are small. - */ -public class BufferGrouper implements Grouper +public class BufferGrouper extends AbstractBufferGrouper { private static final Logger log = new Logger(BufferGrouper.class); - private static final AggregateResult DICTIONARY_FULL = AggregateResult.failure( - "Not enough dictionary space to execute this query. Try increasing " - + "druid.query.groupBy.maxMergingDictionarySize or enable disk spilling by setting " - + "druid.query.groupBy.maxOnDiskStorage to a positive number." - ); - private static final AggregateResult HASHTABLE_FULL = AggregateResult.failure( - "Not enough aggregation table space to execute this query. Try increasing " - + "druid.processing.buffer.sizeBytes or enable disk spilling by setting " - + "druid.query.groupBy.maxOnDiskStorage to a positive number." - ); - private static final int MIN_INITIAL_BUCKETS = 4; private static final int DEFAULT_INITIAL_BUCKETS = 1024; private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f; - private static final int HASH_SIZE = Ints.BYTES; - - private final Supplier bufferSupplier; - private final KeySerde keySerde; - private final int keySize; - private final BufferAggregator[] aggregators; - private final int[] aggregatorOffsets; - private final int initialBuckets; - private final int bucketSize; - private final int bufferGrouperMaxSize; // Integer.MAX_VALUE in production, only used for unit tests - private final float maxLoadFactor; private ByteBuffer buffer; - private int tableArenaSize = -1; - - // Buffer pointing to the current table (it moves around as the table grows) - private ByteBuffer tableBuffer; - - // Offset of tableBuffer within the larger buffer - private int tableStart; - - // Current number of buckets in the table - private int buckets; - - // Number of elements in the table right now - private int size; - - // Maximum number of elements in the table before it must be resized - private int maxSize; - private boolean initialized = false; + // Track the offsets of used buckets using this list. + // When a new bucket is initialized by initializeNewBucketKey(), an offset is added to this list. + // When expanding the table, the list is reset() and filled with the new offsets of the copied buckets. + private ByteBuffer offsetListBuffer; + private ByteBufferIntList offsetList; + public BufferGrouper( final Supplier bufferSupplier, final KeySerde keySerde, @@ -115,12 +61,8 @@ public BufferGrouper( final int initialBuckets ) { - this.bufferSupplier = bufferSupplier; - this.keySerde = keySerde; - this.keySize = keySerde.keySize(); - this.aggregators = new BufferAggregator[aggregatorFactories.length]; - this.aggregatorOffsets = new int[aggregatorFactories.length]; - this.bufferGrouperMaxSize = bufferGrouperMaxSize; + super(bufferSupplier, keySerde, aggregatorFactories, bufferGrouperMaxSize); + this.maxLoadFactor = maxLoadFactor > 0 ? maxLoadFactor : DEFAULT_MAX_LOAD_FACTOR; this.initialBuckets = initialBuckets > 0 ? Math.max(MIN_INITIAL_BUCKETS, initialBuckets) : DEFAULT_INITIAL_BUCKETS; @@ -143,7 +85,38 @@ public void init() { if (!initialized) { this.buffer = bufferSupplier.get(); - this.tableArenaSize = (buffer.capacity() / (bucketSize + Ints.BYTES)) * bucketSize; + + int hashTableSize = ByteBufferHashTable.calculateTableArenaSizeWithPerBucketAdditionalSize( + buffer.capacity(), + bucketSize, + Ints.BYTES + ); + + hashTableBuffer = buffer.duplicate(); + hashTableBuffer.position(0); + hashTableBuffer.limit(hashTableSize); + hashTableBuffer = hashTableBuffer.slice(); + + offsetListBuffer = buffer.duplicate(); + offsetListBuffer.position(hashTableSize); + offsetListBuffer.limit(buffer.capacity()); + offsetListBuffer = offsetListBuffer.slice(); + + this.offsetList = new ByteBufferIntList( + offsetListBuffer, + offsetListBuffer.capacity() / Ints.BYTES + ); + + this.hashTable = new ByteBufferHashTable( + maxLoadFactor, + initialBuckets, + bucketSize, + hashTableBuffer, + keySize, + bufferGrouperMaxSize, + new BufferGrouperBucketUpdateHandler() + ); + reset(); initialized = true; } @@ -156,149 +129,64 @@ public boolean isInitialized() } @Override - public AggregateResult aggregate(KeyType key, int keyHash) + public void newBucketHook(int bucketOffset) { - final ByteBuffer keyBuffer = keySerde.toByteBuffer(key); - if (keyBuffer == null) { - // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will - // be correct. - return DICTIONARY_FULL; - } - - if (keyBuffer.remaining() != keySize) { - throw new IAE( - "keySerde.toByteBuffer(key).remaining[%s] != keySerde.keySize[%s], buffer was the wrong size?!", - keyBuffer.remaining(), - keySize - ); - } - - int bucket = findBucket( - tableBuffer, - buckets, - bucketSize, - size < Math.min(maxSize, bufferGrouperMaxSize), - keyBuffer, - keySize, - keyHash - ); - - if (bucket < 0) { - if (size < bufferGrouperMaxSize) { - growIfPossible(); - bucket = findBucket(tableBuffer, buckets, bucketSize, size < maxSize, keyBuffer, keySize, keyHash); - } - - if (bucket < 0) { - // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will - // be correct. - return HASHTABLE_FULL; - } - } - - final int offset = bucket * bucketSize; - - // Set up key if this is a new bucket. - if (!isUsed(bucket)) { - tableBuffer.position(offset); - tableBuffer.putInt(keyHash | 0x80000000); - tableBuffer.put(keyBuffer); - - for (int i = 0; i < aggregators.length; i++) { - aggregators[i].init(tableBuffer, offset + aggregatorOffsets[i]); - } - - buffer.putInt(tableArenaSize + size * Ints.BYTES, offset); - size++; - } - - // Aggregate the current row. - for (int i = 0; i < aggregators.length; i++) { - aggregators[i].aggregate(tableBuffer, offset + aggregatorOffsets[i]); - } - - return AggregateResult.ok(); } @Override - public AggregateResult aggregate(final KeyType key) + public boolean canSkipAggregate(boolean bucketWasUsed, int bucketOffset) { - return aggregate(key, Groupers.hash(key)); + return false; } @Override - public void reset() + public void afterAggregateHook(int bucketOffset) { - size = 0; - buckets = Math.min(tableArenaSize / bucketSize, initialBuckets); - maxSize = maxSizeForBuckets(buckets); - - if (buckets < 1) { - throw new IAE( - "Not enough capacity for even one row! Need[%,d] but have[%,d].", - bucketSize + Ints.BYTES, - buffer.capacity() - ); - } - - // Start table part-way through the buffer so the last growth can start from zero and thereby use more space. - tableStart = tableArenaSize - buckets * bucketSize; - int nextBuckets = buckets * 2; - while (true) { - final int nextTableStart = tableStart - nextBuckets * bucketSize; - if (nextTableStart > tableArenaSize / 2) { - tableStart = nextTableStart; - nextBuckets = nextBuckets * 2; - } else { - break; - } - } - - if (tableStart < tableArenaSize / 2) { - tableStart = 0; - } - final ByteBuffer bufferDup = buffer.duplicate(); - bufferDup.position(tableStart); - bufferDup.limit(tableStart + buckets * bucketSize); - tableBuffer = bufferDup.slice(); - - // Clear used bits of new table - for (int i = 0; i < buckets; i++) { - tableBuffer.put(i * bucketSize, (byte) 0); - } + } + @Override + public void reset() + { + offsetList.reset(); + hashTable.reset(); keySerde.reset(); } @Override - public Iterator> iterator(final boolean sorted) + public Iterator> iterator(boolean sorted) { + if (!initialized) { + // it's possible for iterator() to be called before initialization when + // a nested groupBy's subquery has an empty result set (see testEmptySubquery() in GroupByQueryRunnerTest) + return Iterators.>emptyIterator(); + } + if (sorted) { final List wrappedOffsets = new AbstractList() { @Override public Integer get(int index) { - return buffer.getInt(tableArenaSize + index * Ints.BYTES); + return offsetList.get(index); } @Override public Integer set(int index, Integer element) { final Integer oldValue = get(index); - buffer.putInt(tableArenaSize + index * Ints.BYTES, element); + offsetList.set(index, element); return oldValue; } @Override public int size() { - return size; + return hashTable.getSize(); } }; - final KeyComparator comparator = keySerde.bufferComparator(); + final BufferComparator comparator = keySerde.bufferComparator(); // Sort offsets in-place. Collections.sort( @@ -308,6 +196,7 @@ public int size() @Override public int compare(Integer lhs, Integer rhs) { + final ByteBuffer tableBuffer = hashTable.getTableBuffer(); return comparator.compare( tableBuffer, tableBuffer, @@ -321,6 +210,7 @@ public int compare(Integer lhs, Integer rhs) return new Iterator>() { int curr = 0; + final int size = getSize(); @Override public boolean hasNext() @@ -331,6 +221,9 @@ public boolean hasNext() @Override public Entry next() { + if (curr >= size) { + throw new NoSuchElementException(); + } return bucketEntryForOffset(wrappedOffsets.get(curr++)); } @@ -345,6 +238,7 @@ public void remove() return new Iterator>() { int curr = 0; + final int size = getSize(); @Override public boolean hasNext() @@ -355,7 +249,10 @@ public boolean hasNext() @Override public Entry next() { - final int offset = buffer.getInt(tableArenaSize + curr * Ints.BYTES); + if (curr >= size) { + throw new NoSuchElementException(); + } + final int offset = offsetList.get(curr); final Entry entry = bucketEntryForOffset(offset); curr++; @@ -371,174 +268,36 @@ public void remove() } } - @Override - public void close() - { - for (BufferAggregator aggregator : aggregators) { - try { - aggregator.close(); - } - catch (Exception e) { - log.warn(e, "Could not close aggregator, skipping.", aggregator); - } - } - } - - private boolean isUsed(final int bucket) - { - return (tableBuffer.get(bucket * bucketSize) & 0x80) == 0x80; - } - - private Entry bucketEntryForOffset(final int bucketOffset) - { - final KeyType key = keySerde.fromByteBuffer(tableBuffer, bucketOffset + HASH_SIZE); - final Object[] values = new Object[aggregators.length]; - for (int i = 0; i < aggregators.length; i++) { - values[i] = aggregators[i].get(tableBuffer, bucketOffset + aggregatorOffsets[i]); - } - - return new Entry<>(key, values); - } - - private void growIfPossible() + private class BufferGrouperBucketUpdateHandler implements ByteBufferHashTable.BucketUpdateHandler { - if (tableStart == 0) { - // tableStart = 0 is the last growth; no further growing is possible. - return; - } - - final int newBuckets; - final int newMaxSize; - final int newTableStart; - - if ((tableStart + buckets * 3 * bucketSize) > tableArenaSize) { - // Not enough space to grow upwards, start back from zero - newTableStart = 0; - newBuckets = tableStart / bucketSize; - newMaxSize = maxSizeForBuckets(newBuckets); - } else { - newTableStart = tableStart + tableBuffer.limit(); - newBuckets = buckets * 2; - newMaxSize = maxSizeForBuckets(newBuckets); - } - - if (newBuckets < buckets) { - throw new ISE("WTF?! newBuckets[%,d] < buckets[%,d]", newBuckets, buckets); - } - - ByteBuffer newTableBuffer = buffer.duplicate(); - newTableBuffer.position(newTableStart); - newTableBuffer.limit(newTableStart + newBuckets * bucketSize); - newTableBuffer = newTableBuffer.slice(); - - int newSize = 0; - - // Clear used bits of new table - for (int i = 0; i < newBuckets; i++) { - newTableBuffer.put(i * bucketSize, (byte) 0); - } - - // Loop over old buckets and copy to new table - final ByteBuffer entryBuffer = tableBuffer.duplicate(); - final ByteBuffer keyBuffer = tableBuffer.duplicate(); - - for (int oldBucket = 0; oldBucket < buckets; oldBucket++) { - if (isUsed(oldBucket)) { - int oldPosition = oldBucket * bucketSize; - entryBuffer.limit((oldBucket + 1) * bucketSize); - entryBuffer.position(oldPosition); - keyBuffer.limit(entryBuffer.position() + HASH_SIZE + keySize); - keyBuffer.position(entryBuffer.position() + HASH_SIZE); - - final int keyHash = entryBuffer.getInt(entryBuffer.position()) & 0x7fffffff; - final int newBucket = findBucket(newTableBuffer, newBuckets, bucketSize, true, keyBuffer, keySize, keyHash); - - if (newBucket < 0) { - throw new ISE("WTF?! Couldn't find a bucket while resizing?!"); - } - - int newPosition = newBucket * bucketSize; - newTableBuffer.position(newPosition); - newTableBuffer.put(entryBuffer); - - for (int i = 0; i < aggregators.length; i++) { - aggregators[i].relocate( - oldPosition + aggregatorOffsets[i], - newPosition + aggregatorOffsets[i], - tableBuffer, - newTableBuffer - ); - } - - buffer.putInt(tableArenaSize + newSize * Ints.BYTES, newBucket * bucketSize); - newSize++; - } + @Override + public void handleNewBucket(int bucketOffset) + { + offsetList.add(bucketOffset); } - buckets = newBuckets; - maxSize = newMaxSize; - tableBuffer = newTableBuffer; - tableStart = newTableStart; - - if (size != newSize) { - throw new ISE("WTF?! size[%,d] != newSize[%,d] after resizing?!", size, maxSize); + @Override + public void handlePreTableSwap() + { + offsetList.reset(); } - } - - private int maxSizeForBuckets(int buckets) - { - return Math.max(1, (int) (buckets * maxLoadFactor)); - } - - /** - * Finds the bucket into which we should insert a key. - * - * @param keyBuffer key, must have exactly keySize bytes remaining. Will not be modified. - * - * @return bucket index for this key, or -1 if no bucket is available due to being full - */ - private static int findBucket( - final ByteBuffer tableBuffer, - final int buckets, - final int bucketSize, - final boolean allowNewBucket, - final ByteBuffer keyBuffer, - final int keySize, - final int keyHash - ) - { - // startBucket will never be negative since keyHash is always positive (see Groupers.hash) - final int startBucket = keyHash % buckets; - int bucket = startBucket; - -outer: - while (true) { - final int bucketOffset = bucket * bucketSize; - - if ((tableBuffer.get(bucketOffset) & 0x80) == 0) { - // Found unused bucket before finding our key - return allowNewBucket ? bucket : -1; - } - - for (int i = bucketOffset + HASH_SIZE, j = keyBuffer.position(); j < keyBuffer.position() + keySize; i++, j++) { - if (tableBuffer.get(i) != keyBuffer.get(j)) { - bucket += 1; - if (bucket == buckets) { - bucket = 0; - } - if (bucket == startBucket) { - // Came back around to the start without finding a free slot, that was a long trip! - // Should never happen unless buckets == maxSize. - return -1; - } - - continue outer; - } + @Override + public void handleBucketMove( + int oldBucketOffset, int newBucketOffset, ByteBuffer oldBuffer, ByteBuffer newBuffer + ) + { + // relocate aggregators (see https://github.com/druid-io/druid/pull/4071) + for (int i = 0; i < aggregators.length; i++) { + aggregators[i].relocate( + oldBucketOffset + aggregatorOffsets[i], + newBucketOffset + aggregatorOffsets[i], + oldBuffer, + newBuffer + ); } - // Found our key in a used bucket - return bucket; + offsetList.add(newBucketOffset); } } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java new file mode 100644 index 000000000000..cd83b229c355 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java @@ -0,0 +1,382 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.primitives.Ints; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; + +import java.nio.ByteBuffer; + +public class ByteBufferHashTable +{ + public static int calculateTableArenaSizeWithPerBucketAdditionalSize( + int bufferCapacity, + int bucketSize, + int perBucketAdditionalSize + ) + { + return (bufferCapacity / (bucketSize + perBucketAdditionalSize)) * bucketSize; + } + + public static int calculateTableArenaSizeWithFixedAdditionalSize( + int bufferCapacity, + int bucketSize, + int fixedAdditionalSize + ) + { + return ((bufferCapacity - fixedAdditionalSize) / bucketSize) * bucketSize; + } + + protected final int maxSizeForTesting; // Integer.MAX_VALUE in production, only used for unit tests + + protected static final int HASH_SIZE = Ints.BYTES; + + protected final float maxLoadFactor; + protected final int initialBuckets; + protected final ByteBuffer buffer; + protected final int bucketSizeWithHash; + protected final int tableArenaSize; + protected final int keySize; + + protected int tableStart; + + // Buffer pointing to the current table (it moves around as the table grows) + protected ByteBuffer tableBuffer; + + // Number of elements in the table right now + protected int size; + + // Maximum number of elements in the table before it must be resized + // This value changes when the table is resized. + protected int regrowthThreshold; + + // current number of available/used buckets in the table + // This value changes when the table is resized. + protected int maxBuckets; + + // how many times the table buffer has filled/readjusted (through adjustTableWhenFull()) + protected int growthCount; + + + + protected BucketUpdateHandler bucketUpdateHandler; + + public ByteBufferHashTable( + float maxLoadFactor, + int initialBuckets, + int bucketSizeWithHash, + ByteBuffer buffer, + int keySize, + int maxSizeForTesting, + BucketUpdateHandler bucketUpdateHandler + ) + { + this.maxLoadFactor = maxLoadFactor; + this.initialBuckets = initialBuckets; + this.bucketSizeWithHash = bucketSizeWithHash; + this.buffer = buffer; + this.keySize = keySize; + this.maxSizeForTesting = maxSizeForTesting; + this.tableArenaSize = buffer.capacity(); + this.bucketUpdateHandler = bucketUpdateHandler; + } + + public void reset() + { + size = 0; + + maxBuckets = Math.min(tableArenaSize / bucketSizeWithHash, initialBuckets); + regrowthThreshold = maxSizeForBuckets(maxBuckets); + + if (maxBuckets < 1) { + throw new IAE( + "Not enough capacity for even one row! Need[%,d] but have[%,d].", + bucketSizeWithHash + Ints.BYTES, + buffer.capacity() + ); + } + + // Start table part-way through the buffer so the last growth can start from zero and thereby use more space. + tableStart = tableArenaSize - maxBuckets * bucketSizeWithHash; + int nextBuckets = maxBuckets * 2; + while (true) { + final int nextTableStart = tableStart - nextBuckets * bucketSizeWithHash; + if (nextTableStart > tableArenaSize / 2) { + tableStart = nextTableStart; + nextBuckets = nextBuckets * 2; + } else { + break; + } + } + + if (tableStart < tableArenaSize / 2) { + tableStart = 0; + } + + final ByteBuffer bufferDup = buffer.duplicate(); + bufferDup.position(tableStart); + bufferDup.limit(tableStart + maxBuckets * bucketSizeWithHash); + tableBuffer = bufferDup.slice(); + + // Clear used bits of new table + for (int i = 0; i < maxBuckets; i++) { + tableBuffer.put(i * bucketSizeWithHash, (byte) 0); + } + } + + public void adjustTableWhenFull() + { + if (tableStart == 0) { + // tableStart = 0 is the last growth; no further growing is possible. + return; + } + + final int newBuckets; + final int newMaxSize; + final int newTableStart; + + if (((long) maxBuckets * 3 * bucketSizeWithHash) > (long) tableArenaSize - tableStart) { + // Not enough space to grow upwards, start back from zero + newTableStart = 0; + newBuckets = tableStart / bucketSizeWithHash; + newMaxSize = maxSizeForBuckets(newBuckets); + } else { + newTableStart = tableStart + tableBuffer.limit(); + newBuckets = maxBuckets * 2; + newMaxSize = maxSizeForBuckets(newBuckets); + } + + if (newBuckets < maxBuckets) { + throw new ISE("WTF?! newBuckets[%,d] < maxBuckets[%,d]", newBuckets, maxBuckets); + } + + ByteBuffer newTableBuffer = buffer.duplicate(); + newTableBuffer.position(newTableStart); + newTableBuffer.limit(newTableStart + newBuckets * bucketSizeWithHash); + newTableBuffer = newTableBuffer.slice(); + + int newSize = 0; + + // Clear used bits of new table + for (int i = 0; i < newBuckets; i++) { + newTableBuffer.put(i * bucketSizeWithHash, (byte) 0); + } + + // Loop over old buckets and copy to new table + final ByteBuffer entryBuffer = tableBuffer.duplicate(); + final ByteBuffer keyBuffer = tableBuffer.duplicate(); + + int oldBuckets = maxBuckets; + + if (bucketUpdateHandler != null) { + bucketUpdateHandler.handlePreTableSwap(); + } + + for (int oldBucket = 0; oldBucket < oldBuckets; oldBucket++) { + if (isBucketUsed(oldBucket)) { + int oldBucketOffset = oldBucket * bucketSizeWithHash; + entryBuffer.limit((oldBucket + 1) * bucketSizeWithHash); + entryBuffer.position(oldBucketOffset); + keyBuffer.limit(entryBuffer.position() + HASH_SIZE + keySize); + keyBuffer.position(entryBuffer.position() + HASH_SIZE); + + final int keyHash = entryBuffer.getInt(entryBuffer.position()) & 0x7fffffff; + final int newBucket = findBucket(true, newBuckets, newTableBuffer, keyBuffer, keyHash); + + if (newBucket < 0) { + throw new ISE("WTF?! Couldn't find a bucket while resizing?!"); + } + + final int newBucketOffset = newBucket * bucketSizeWithHash; + + newTableBuffer.position(newBucketOffset); + newTableBuffer.put(entryBuffer); + + newSize++; + + if (bucketUpdateHandler != null) { + bucketUpdateHandler.handleBucketMove(oldBucketOffset, newBucketOffset, tableBuffer, newTableBuffer); + } + } + } + + maxBuckets = newBuckets; + regrowthThreshold = newMaxSize; + tableBuffer = newTableBuffer; + tableStart = newTableStart; + + growthCount++; + + if (size != newSize) { + throw new ISE("WTF?! size[%,d] != newSize[%,d] after resizing?!", size, newSize); + } + } + + protected void initializeNewBucketKey( + final int bucket, + final ByteBuffer keyBuffer, + final int keyHash + ) + { + int offset = bucket * bucketSizeWithHash; + tableBuffer.position(offset); + tableBuffer.putInt(keyHash | 0x80000000); + tableBuffer.put(keyBuffer); + size++; + + if (bucketUpdateHandler != null) { + bucketUpdateHandler.handleNewBucket(offset); + } + } + + /** + * Find a bucket for a key, attempting to resize the table with adjustTableWhenFull() if possible. + * + * @param keyBuffer buffer containing the key + * @param keyHash hash of the key + * @return bucket number of the found bucket or -1 if a bucket could not be allocated after resizing. + */ + protected int findBucketWithAutoGrowth( + final ByteBuffer keyBuffer, + final int keyHash + ) + { + int bucket = findBucket(canAllowNewBucket(), maxBuckets, tableBuffer, keyBuffer, keyHash); + + if (bucket < 0) { + if (size < maxSizeForTesting) { + adjustTableWhenFull(); + bucket = findBucket(size < regrowthThreshold, maxBuckets, tableBuffer, keyBuffer, keyHash); + } + } + + return bucket; + } + + /** + * Finds the bucket into which we should insert a key. + * + * @param keyBuffer key, must have exactly keySize bytes remaining. Will not be modified. + * @param targetTableBuffer Need selectable buffer, since when resizing hash table, + * findBucket() is used on the newly allocated table buffer + * + * @return bucket index for this key, or -1 if no bucket is available due to being full + */ + protected int findBucket( + final boolean allowNewBucket, + final int buckets, + final ByteBuffer targetTableBuffer, + final ByteBuffer keyBuffer, + final int keyHash + ) + { + // startBucket will never be negative since keyHash is always positive (see Groupers.hash) + final int startBucket = keyHash % buckets; + int bucket = startBucket; + +outer: + while (true) { + final int bucketOffset = bucket * bucketSizeWithHash; + + if ((targetTableBuffer.get(bucketOffset) & 0x80) == 0) { + // Found unused bucket before finding our key + return allowNewBucket ? bucket : -1; + } + + for (int i = bucketOffset + HASH_SIZE, j = keyBuffer.position(); j < keyBuffer.position() + keySize; i++, j++) { + if (targetTableBuffer.get(i) != keyBuffer.get(j)) { + bucket += 1; + if (bucket == buckets) { + bucket = 0; + } + + if (bucket == startBucket) { + // Came back around to the start without finding a free slot, that was a long trip! + // Should never happen unless buckets == regrowthThreshold. + return -1; + } + + continue outer; + } + } + + // Found our key in a used bucket + return bucket; + } + } + + protected boolean canAllowNewBucket() + { + return size < Math.min(regrowthThreshold, maxSizeForTesting); + } + + protected int getOffsetForBucket(int bucket) + { + return bucket * bucketSizeWithHash; + } + + protected int maxSizeForBuckets(int buckets) + { + return Math.max(1, (int) (buckets * maxLoadFactor)); + } + + protected boolean isBucketUsed(final int bucket) + { + return (tableBuffer.get(bucket * bucketSizeWithHash) & 0x80) == 0x80; + } + + protected boolean isOffsetUsed(final int bucketOffset) + { + return (tableBuffer.get(bucketOffset) & 0x80) == 0x80; + } + + public ByteBuffer getTableBuffer() + { + return tableBuffer; + } + + public int getSize() + { + return size; + } + + public int getRegrowthThreshold() + { + return regrowthThreshold; + } + + public int getMaxBuckets() + { + return maxBuckets; + } + + public int getGrowthCount() + { + return growthCount; + } + + public interface BucketUpdateHandler + { + void handleNewBucket(int bucketOffset); + void handlePreTableSwap(); + void handleBucketMove(int oldBucketOffset, int newBucketOffset, ByteBuffer oldBuffer, ByteBuffer newBuffer); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferIntList.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferIntList.java new file mode 100644 index 000000000000..2fe1706b2a24 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferIntList.java @@ -0,0 +1,78 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.primitives.Ints; +import com.metamx.common.IAE; + +import java.nio.ByteBuffer; + +public class ByteBufferIntList +{ + private final ByteBuffer buffer; + private final int maxElements; + private int numElements; + + public ByteBufferIntList( + ByteBuffer buffer, + int maxElements + ) + { + this.buffer = buffer; + this.maxElements = maxElements; + this.numElements = 0; + + if (buffer.capacity() < (maxElements * Ints.BYTES)) { + throw new IAE( + "buffer for list is too small, was [%s] bytes, but need [%s] bytes.", + buffer.capacity(), + maxElements * Ints.BYTES + ); + } + } + + public void add(int val) + { + if (numElements == maxElements) { + throw new IndexOutOfBoundsException(String.format("List is full with %s elements.", maxElements)); + } + buffer.putInt(numElements * Ints.BYTES, val); + numElements++; + } + + public void set(int index, int val) + { + buffer.putInt(index * Ints.BYTES, val); + } + + public int get(int index) { + return buffer.getInt(index * Ints.BYTES); + } + + public int getNumElements() + { + return numElements; + } + + public void reset() + { + numElements = 0; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java new file mode 100644 index 000000000000..ea203a660514 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeap.java @@ -0,0 +1,493 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Ordering; +import com.google.common.primitives.Ints; +import io.druid.java.util.common.ISE; + +import java.nio.ByteBuffer; +import java.util.Comparator; + +/** + * ByteBuffer-based implementation of the min-max heap developed by Atkinson, et al. + * (http://portal.acm.org/citation.cfm?id=6621), with some utility functions from + * Guava's MinMaxPriorityQueue. + */ +public class ByteBufferMinMaxOffsetHeap +{ + private static final int EVEN_POWERS_OF_TWO = 0x55555555; + private static final int ODD_POWERS_OF_TWO = 0xaaaaaaaa; + + private final Comparator minComparator; + private final Comparator maxComparator; + private final ByteBuffer buf; + private final int limit; + private final LimitedBufferGrouper.BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater; + + private int heapSize; + + public ByteBufferMinMaxOffsetHeap( + ByteBuffer buf, + int limit, + Comparator minComparator, + LimitedBufferGrouper.BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater + ) + { + this.buf = buf; + this.limit = limit; + this.heapSize = 0; + this.minComparator = minComparator; + this.maxComparator = Ordering.from(minComparator).reverse(); + this.heapIndexUpdater = heapIndexUpdater; + } + + public void reset() + { + heapSize = 0; + } + + public int addOffset(int offset) + { + int pos = heapSize; + buf.putInt(pos * Ints.BYTES, offset); + heapSize++; + + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(offset, pos); + } + + bubbleUp(pos); + + if (heapSize > limit) { + return removeMax(); + } else { + return -1; + } + } + + public int removeMin() { + if (heapSize < 1) { + throw new ISE("Empty heap"); + } + int minOffset = buf.getInt(0); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(minOffset, -1); + } + + if (heapSize == 1) { + heapSize--; + return minOffset; + } + + int lastIndex = heapSize - 1; + int lastOffset = buf.getInt(lastIndex * Ints.BYTES); + heapSize--; + buf.putInt(0, lastOffset); + + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(lastOffset, 0); + } + + Comparator comparator = isEvenLevel(0) ? minComparator : maxComparator; + siftDown(comparator, 0); + + return minOffset; + } + + public int removeMax() { + int maxOffset; + if (heapSize < 1) { + throw new ISE("Empty heap"); + } + if (heapSize == 1) { + heapSize--; + maxOffset = buf.getInt(0); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(maxOffset, -1); + } + return maxOffset; + } + + // index of max must be 1, just remove it and shrink the heap + if (heapSize == 2) { + heapSize--; + maxOffset = buf.getInt(Ints.BYTES); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(maxOffset, -1); + } + return maxOffset; + } + + int maxIndex = findMaxElementIndex(); + maxOffset = buf.getInt(maxIndex * Ints.BYTES); + + int lastIndex = heapSize - 1; + int lastOffset = buf.getInt(lastIndex * Ints.BYTES); + heapSize--; + buf.putInt(maxIndex * Ints.BYTES, lastOffset); + + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(maxOffset, -1); + heapIndexUpdater.updateHeapIndexForOffset(lastOffset, maxIndex); + } + + Comparator comparator = isEvenLevel(maxIndex) ? minComparator : maxComparator; + siftDown(comparator, maxIndex); + + return maxOffset; + } + + public int removeAt(int deletedIndex) { + if (heapSize < 1) { + throw new ISE("Empty heap"); + } + int deletedOffset = buf.getInt(deletedIndex * Ints.BYTES); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(deletedOffset, -1); + } + + int lastIndex = heapSize - 1; + heapSize--; + if (lastIndex == deletedIndex) { + return deletedOffset; + } + int lastOffset = buf.getInt(lastIndex * Ints.BYTES); + buf.putInt(deletedIndex * Ints.BYTES, lastOffset); + + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(lastOffset, deletedIndex); + } + + Comparator comparator = isEvenLevel(deletedIndex) ? minComparator : maxComparator; + + bubbleUp(deletedIndex); + siftDown(comparator, deletedIndex); + + return deletedOffset; + } + + public void setAt(int index, int newVal) { + buf.putInt(index * Ints.BYTES, newVal); + } + + public int getAt(int index) { + return buf.getInt(index * Ints.BYTES); + } + + public int indexOf(int offset) { + for (int i = 0; i < heapSize; i++) { + int curOffset = buf.getInt(i * Ints.BYTES); + if (curOffset == offset) { + return i; + } + } + return -1; + } + + public void removeOffset(int offset) { + int index = indexOf(offset); + if (index > -1) { + removeAt(index); + } + } + + public int getHeapSize() { + return heapSize; + } + + private void bubbleUp(int pos) + { + if (isEvenLevel(pos)) { + int parentIndex = getParentIndex(pos); + if (parentIndex > -1) { + int parentOffset = buf.getInt(parentIndex * Ints.BYTES); + int offset = buf.getInt(pos * Ints.BYTES); + if (minComparator.compare(offset, parentOffset) > 0) { + buf.putInt(parentIndex * Ints.BYTES, offset); + buf.putInt(pos * Ints.BYTES, parentOffset); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(offset, parentIndex); + heapIndexUpdater.updateHeapIndexForOffset(parentOffset, pos); + } + bubbleUpDirectional(maxComparator, parentIndex); + } else { + bubbleUpDirectional(minComparator, pos); + } + } else { + bubbleUpDirectional(minComparator, pos); + } + } else { + int parentIndex = getParentIndex(pos); + if (parentIndex > -1) { + int parentOffset = buf.getInt(parentIndex * Ints.BYTES); + int offset = buf.getInt(pos * Ints.BYTES); + if (minComparator.compare(offset, parentOffset) < 0) { + buf.putInt(parentIndex * Ints.BYTES, offset); + buf.putInt(pos * Ints.BYTES, parentOffset); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(offset, parentIndex); + heapIndexUpdater.updateHeapIndexForOffset(parentOffset, pos); + } + bubbleUpDirectional(minComparator, parentIndex); + } else { + bubbleUpDirectional(maxComparator, pos); + } + } else { + bubbleUpDirectional(maxComparator, pos); + } + } + } + + private void bubbleUpDirectional(Comparator comparator, int pos) + { + int grandparent = getGrandparentIndex(pos); + while (grandparent > -1) { + int offset = buf.getInt(pos * Ints.BYTES); + int gpOffset = buf.getInt(grandparent * Ints.BYTES); + + if (comparator.compare(offset, gpOffset) < 0) { + buf.putInt(pos * Ints.BYTES, gpOffset); + buf.putInt(grandparent * Ints.BYTES, offset); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(gpOffset, pos); + heapIndexUpdater.updateHeapIndexForOffset(offset, grandparent); + } + } + pos = grandparent; + grandparent = getGrandparentIndex(pos); + } + } + + private void siftDown(Comparator comparator, int pos) + { + int minChild = findMinChild(comparator, pos); + int minGrandchild; + int minIndex; + while (minChild > -1) { + minGrandchild = findMinGrandChild(comparator, pos); + if (minGrandchild > -1) { + int minChildOffset = buf.getInt(minChild * Ints.BYTES); + int minGcOffset = buf.getInt(minGrandchild * Ints.BYTES); + int cmp = comparator.compare(minChildOffset, minGcOffset); + minIndex = (cmp > 0) ? minGrandchild : minChild; + } else if (minChild > -1) { + minIndex = minChild; + } else { + break; + } + if (minIndex == minGrandchild) { + int offset = buf.getInt(pos * Ints.BYTES); + int minOffset = buf.getInt(minIndex * Ints.BYTES); + + if (comparator.compare(minOffset, offset) < 0) { + buf.putInt(pos * Ints.BYTES, minOffset); + buf.putInt(minIndex * Ints.BYTES, offset); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(minOffset, pos); + heapIndexUpdater.updateHeapIndexForOffset(offset, minIndex); + } + + int parent = getParentIndex(minIndex); + int parentOffset = buf.getInt(parent * Ints.BYTES); + + if (comparator.compare(offset, parentOffset) > 0) { + buf.putInt(minIndex * Ints.BYTES, parentOffset); + buf.putInt(parent * Ints.BYTES, offset); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(offset, parent); + heapIndexUpdater.updateHeapIndexForOffset(parentOffset, minIndex); + } + } + minChild = findMinChild(comparator, minIndex); + } + pos = minIndex; + } else { + int offset = buf.getInt(pos * Ints.BYTES); + int minOffset = buf.getInt(minIndex * Ints.BYTES); + if (comparator.compare(minOffset, offset) < 0) { + buf.putInt(pos * Ints.BYTES, minOffset); + buf.putInt(minIndex * Ints.BYTES, offset); + if (heapIndexUpdater != null) { + heapIndexUpdater.updateHeapIndexForOffset(offset, minIndex); + heapIndexUpdater.updateHeapIndexForOffset(minOffset, pos); + } + } + break; + } + } + } + + private boolean isEvenLevel(int index) { + int oneBased = index + 1; + return (oneBased & EVEN_POWERS_OF_TWO) > (oneBased & ODD_POWERS_OF_TWO); + } + + /** + * Returns the index of minimum value between {@code index} and + * {@code index + len}, or {@code -1} if {@code index} is greater than + * {@code size}. + */ + private int findMin(Comparator comparator, int index, int len) { + if (index >= heapSize) { + return -1; + } + int limit = Math.min(index, heapSize - len) + len; + int minIndex = index; + for (int i = index + 1; i < limit; i++) { + if (comparator.compare(buf.getInt(i * Ints.BYTES), buf.getInt(minIndex * Ints.BYTES)) < 0) { + minIndex = i; + } + } + return minIndex; + } + + /** + * Returns the minimum child or {@code -1} if no child exists. + */ + private int findMinChild(Comparator comparator, int index) { + return findMin(comparator, getLeftChildIndex(index), 2); + } + + /** + * Returns the minimum grand child or -1 if no grand child exists. + */ + private int findMinGrandChild(Comparator comparator, int index) { + int leftChildIndex = getLeftChildIndex(index); + if (leftChildIndex < 0) { + return -1; + } + return findMin(comparator, getLeftChildIndex(leftChildIndex), 4); + } + + private int getLeftChildIndex(int i) { + return i * 2 + 1; + } + + private int getRightChildIndex(int i) { + return i * 2 + 2; + } + + private int getParentIndex(int i) { + if (i == 0) { + return -1; + } + return (i - 1) / 2; + } + + private int getGrandparentIndex(int i) { + if (i < 3) { + return -1; + } + return (i - 3) / 4; + } + + /** + * Returns the index of the max element. + */ + private int findMaxElementIndex() { + switch (heapSize) { + case 1: + return 0; // The lone element in the queue is the maximum. + case 2: + return 1; // The lone element in the maxHeap is the maximum. + default: + // The max element must sit on the first level of the maxHeap. It is + // actually the *lesser* of the two from the maxHeap's perspective. + int offset1 = buf.getInt(1 * Ints.BYTES); + int offset2 = buf.getInt(2 * Ints.BYTES); + return maxComparator.compare(offset1, offset2) <= 0 ? 1 : 2; + } + } + + @VisibleForTesting + boolean isIntact() { + for (int i = 0; i < heapSize; i++) { + if (!verifyIndex(i)) { + return false; + } + } + return true; + } + + private boolean verifyIndex(int i) + { + Comparator comparator = isEvenLevel(i) ? minComparator : maxComparator; + int offset = buf.getInt(i * Ints.BYTES); + + int lcIdx = getLeftChildIndex(i); + if (lcIdx < heapSize) { + int leftChildOffset = buf.getInt(lcIdx * Ints.BYTES); + if (comparator.compare(offset, leftChildOffset) > 0) { + throw new ISE("Left child val[%d] at idx[%d] is less than val[%d] at idx[%d]", + leftChildOffset, lcIdx, offset, i); + } + } + + int rcIdx = getRightChildIndex(i); + if (rcIdx < heapSize) { + int rightChildOffset = buf.getInt(rcIdx * Ints.BYTES); + if (comparator.compare(offset, rightChildOffset) > 0) { + throw new ISE("Right child val[%d] at idx[%d] is less than val[%d] at idx[%d]", + rightChildOffset, rcIdx, offset, i); + } + } + + if (i > 0) { + int parentIdx = getParentIndex(i); + int parentOffset = buf.getInt(parentIdx * Ints.BYTES); + if (comparator.compare(offset, parentOffset) > 0) { + throw new ISE("Parent val[%d] at idx[%d] is less than val[%d] at idx[%d]", + parentOffset, parentIdx, offset, i); + } + } + + if (i > 2) { + int gpIdx = getGrandparentIndex(i); + int gpOffset = buf.getInt(gpIdx * Ints.BYTES); + if (comparator.compare(gpOffset, offset) > 0) { + throw new ISE("Grandparent val[%d] at idx[%d] is less than val[%d] at idx[%d]", + gpOffset, gpIdx, offset, i); + } + } + + return true; + } + + @Override + public String toString() + { + if (heapSize == 0) { + return "[]"; + } + + String ret = "["; + for (int i = 0; i < heapSize; i++) { + ret += buf.getInt(i * Ints.BYTES); + if (i < heapSize - 1) { + ret += ", "; + } + } + + ret += "]"; + return ret; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java index aa4ee9a73c70..71d3ff6f1543 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java @@ -25,6 +25,7 @@ import com.google.common.base.Suppliers; import io.druid.java.util.common.ISE; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.segment.ColumnSelectorFactory; import java.nio.ByteBuffer; @@ -50,7 +51,7 @@ public class ConcurrentGrouper implements Grouper private final AtomicInteger threadNumber = new AtomicInteger(); private volatile boolean spilling = false; private volatile boolean closed = false; - private final Comparator keyObjComparator; + private final Comparator> keyObjComparator; private final Supplier bufferSupplier; private final ColumnSelectorFactory columnSelectorFactory; @@ -62,6 +63,8 @@ public class ConcurrentGrouper implements Grouper private final ObjectMapper spillMapper; private final int concurrencyHint; private final KeySerdeFactory keySerdeFactory; + private final DefaultLimitSpec limitSpec; + private final boolean sortHasNonGroupingFields; private volatile boolean initialized = false; @@ -75,7 +78,9 @@ public ConcurrentGrouper( final int bufferGrouperInitialBuckets, final LimitedTemporaryStorage temporaryStorage, final ObjectMapper spillMapper, - final int concurrencyHint + final int concurrencyHint, + final DefaultLimitSpec limitSpec, + final boolean sortHasNonGroupingFields ) { Preconditions.checkArgument(concurrencyHint > 0, "concurrencyHint > 0"); @@ -100,7 +105,9 @@ protected SpillingGrouper initialValue() this.spillMapper = spillMapper; this.concurrencyHint = concurrencyHint; this.keySerdeFactory = keySerdeFactory; - this.keyObjComparator = keySerdeFactory.objectComparator(); + this.limitSpec = limitSpec; + this.sortHasNonGroupingFields = sortHasNonGroupingFields; + this.keyObjComparator = keySerdeFactory.objectComparator(sortHasNonGroupingFields); } @Override @@ -126,7 +133,9 @@ public void init() bufferGrouperInitialBuckets, temporaryStorage, spillMapper, - false + false, + limitSpec, + sortHasNonGroupingFields ); grouper.init(); groupers.add(grouper); diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index de6fee13b21a..c7b09faee993 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -45,9 +45,9 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.ChainedExecutionQueryRunner; -import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryWatcher; import io.druid.query.ResourceLimitExceededException; @@ -105,9 +105,9 @@ public GroupByMergingQueryRunnerV2( } @Override - public Sequence run(final Query queryParam, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - final GroupByQuery query = (GroupByQuery) queryParam; + final GroupByQuery query = (GroupByQuery) queryPlus.getQuery(); final GroupByQueryConfig querySpecificConfig = config.withOverrides(query); // CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION is here because realtime servers use nested mergeRunners calls @@ -119,12 +119,15 @@ public Sequence run(final Query queryParam, final Map CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, false ); - final GroupByQuery queryForRunners = query.withOverriddenContext( - ImmutableMap.of(CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, true) - ); + final QueryPlus queryPlusForRunners = queryPlus + .withQuery( + query.withOverriddenContext(ImmutableMap.of(CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION, true)) + ) + .withoutThreadUnsafeState(); if (QueryContexts.isBySegment(query) || forceChainedExecution) { - return new ChainedExecutionQueryRunner(exec, queryWatcher, queryables).run(query, responseContext); + ChainedExecutionQueryRunner runner = new ChainedExecutionQueryRunner<>(exec, queryWatcher, queryables); + return runner.run(queryPlusForRunners, responseContext); } final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded(); @@ -225,7 +228,7 @@ public AggregateResult call() throws Exception Releaser bufferReleaser = mergeBufferHolder.increment(); Releaser grouperReleaser = grouperHolder.increment() ) { - final AggregateResult retVal = input.run(queryForRunners, responseContext) + final AggregateResult retVal = input.run(queryPlusForRunners, responseContext) .accumulate( AggregateResult.ok(), accumulator diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java index a0f82b996304..932136f07513 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -441,12 +441,21 @@ public ByteBuffer fromByteBuffer(ByteBuffer buffer, int position) } @Override - public Grouper.KeyComparator bufferComparator() + public Grouper.BufferComparator bufferComparator() { // No sorting, let mergeRunners handle that throw new UnsupportedOperationException(); } + @Override + public Grouper.BufferComparator bufferComparatorWithAggregators( + AggregatorFactory[] aggregatorFactories, int[] aggregatorOffsets + ) + { + // not called on this + throw new UnsupportedOperationException(); + } + @Override public void reset() { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java index 1971c2b86b27..2f6f795fd091 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.aggregation.AggregatorFactory; import java.io.Closeable; import java.nio.ByteBuffer; @@ -178,9 +179,12 @@ interface KeySerdeFactory * Return an object that knows how to compare two serialized key instances. Will be called by the * {@link #iterator(boolean)} method if sorting is enabled. * + * @param forceDefaultOrder Return a comparator that sorts by the key in default lexicographic ascending order, + * regardless of any other conditions (e.g., presence of OrderBySpecs). + * * @return comparator for key objects. */ - Comparator objectComparator(); + Comparator> objectComparator(boolean forceDefaultOrder); } /** @@ -228,7 +232,19 @@ interface KeySerde * * @return comparator for keys */ - KeyComparator bufferComparator(); + BufferComparator bufferComparator(); + + /** + * When pushing down limits, it may also be necessary to compare aggregated values along with the key + * using the bufferComparator. + * + * @param aggregatorFactories Array of aggregators from a GroupByQuery + * @param aggregatorOffsets Offsets for each aggregator in aggregatorFactories pointing to their location + * within the grouping key + aggs buffer. + * + * @return comparator for keys + aggs + */ + BufferComparator bufferComparatorWithAggregators(AggregatorFactory[] aggregatorFactories, int[] aggregatorOffsets); /** * Reset the keySerde to its initial state. After this method is called, {@link #fromByteBuffer(ByteBuffer, int)} @@ -237,7 +253,7 @@ interface KeySerde void reset(); } - interface KeyComparator + interface BufferComparator { int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java index 2990a07548e5..2324f0f3779d 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Groupers.java @@ -34,7 +34,7 @@ private Groupers() private static final int C1 = 0xcc9e2d51; private static final int C2 = 0x1b873593; - /* + /** * This method was rewritten in Java from an intermediate step of the Murmur hash function in * https://github.com/aappleby/smhasher/blob/master/src/MurmurHash3.cpp, which contained the * following header: @@ -57,7 +57,7 @@ public static int hash(final Object obj) public static Iterator> mergeIterators( final Iterable>> iterators, - final Comparator keyTypeComparator + final Comparator> keyTypeComparator ) { if (keyTypeComparator != null) { @@ -68,7 +68,7 @@ public static Iterator> mergeIterators( @Override public int compare(Grouper.Entry lhs, Grouper.Entry rhs) { - return keyTypeComparator.compare(lhs.getKey(), rhs.getKey()); + return keyTypeComparator.compare(lhs, rhs); } } ); diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java new file mode 100644 index 000000000000..dd6442835bab --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouper.java @@ -0,0 +1,519 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Supplier; +import com.google.common.collect.Iterators; +import com.google.common.primitives.Ints; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.AbstractList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +public class LimitedBufferGrouper extends AbstractBufferGrouper +{ + private static final int MIN_INITIAL_BUCKETS = 4; + private static final int DEFAULT_INITIAL_BUCKETS = 1024; + private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f; + + private final AggregatorFactory[] aggregatorFactories; + + // Limit to apply to results. + private int limit; + + // Indicates if the sorting order has fields not in the grouping key, used when pushing down limit/sorting. + // In this case, grouping key comparisons need to also compare on aggregators. + // Additionally, results must be resorted by grouping key to allow results to merge correctly. + private boolean sortHasNonGroupingFields; + + // Min-max heap, used for storing offsets when applying limits/sorting in the BufferGrouper + private ByteBufferMinMaxOffsetHeap offsetHeap; + + // ByteBuffer slices used by the grouper + private ByteBuffer totalBuffer; + private ByteBuffer hashTableBuffer; + private ByteBuffer offsetHeapBuffer; + + // Updates the heap index field for buckets, created passed to the heap when + // pushing down limit and the sort order includes aggregators + private BufferGrouperOffsetHeapIndexUpdater heapIndexUpdater; + private boolean initialized = false; + + public LimitedBufferGrouper( + final Supplier bufferSupplier, + final Grouper.KeySerde keySerde, + final ColumnSelectorFactory columnSelectorFactory, + final AggregatorFactory[] aggregatorFactories, + final int bufferGrouperMaxSize, + final float maxLoadFactor, + final int initialBuckets, + final int limit, + final boolean sortHasNonGroupingFields + ) + { + super(bufferSupplier, keySerde, aggregatorFactories, bufferGrouperMaxSize); + this.maxLoadFactor = maxLoadFactor > 0 ? maxLoadFactor : DEFAULT_MAX_LOAD_FACTOR; + this.initialBuckets = initialBuckets > 0 ? Math.max(MIN_INITIAL_BUCKETS, initialBuckets) : DEFAULT_INITIAL_BUCKETS; + this.limit = limit; + this.sortHasNonGroupingFields = sortHasNonGroupingFields; + + if (this.maxLoadFactor >= 1.0f) { + throw new IAE("Invalid maxLoadFactor[%f], must be < 1.0", maxLoadFactor); + } + + int offset = HASH_SIZE + keySize; + this.aggregatorFactories = aggregatorFactories; + for (int i = 0; i < aggregatorFactories.length; i++) { + aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory); + aggregatorOffsets[i] = offset; + offset += aggregatorFactories[i].getMaxIntermediateSize(); + } + + // For each bucket, store an extra field indicating the bucket's current index within the heap when + // pushing down limits + offset += Ints.BYTES; + this.bucketSize = offset; + } + + @Override + public void init() + { + if (initialized) { + return; + } + this.totalBuffer = bufferSupplier.get(); + + validateBufferCapacity( + limit, + maxLoadFactor, + totalBuffer, + bucketSize + ); + + //only store offsets up to `limit` + 1 instead of up to # of buckets, we only keep the top results + int heapByteSize = (limit + 1) * Ints.BYTES; + + int hashTableSize = ByteBufferHashTable.calculateTableArenaSizeWithFixedAdditionalSize( + totalBuffer.capacity(), + bucketSize, + heapByteSize + ); + + hashTableBuffer = totalBuffer.duplicate(); + hashTableBuffer.position(0); + hashTableBuffer.limit(hashTableSize); + hashTableBuffer = hashTableBuffer.slice(); + + offsetHeapBuffer = totalBuffer.duplicate(); + offsetHeapBuffer.position(hashTableSize); + offsetHeapBuffer = offsetHeapBuffer.slice(); + offsetHeapBuffer.limit(totalBuffer.capacity() - hashTableSize); + + this.hashTable = new AlternatingByteBufferHashTable( + maxLoadFactor, + initialBuckets, + bucketSize, + hashTableBuffer, + keySize, + bufferGrouperMaxSize + ); + this.heapIndexUpdater = new BufferGrouperOffsetHeapIndexUpdater(totalBuffer, bucketSize - Ints.BYTES); + this.offsetHeap = new ByteBufferMinMaxOffsetHeap(offsetHeapBuffer, limit, makeHeapComparator(), heapIndexUpdater); + + reset(); + + initialized = true; + } + + @Override + public boolean isInitialized() + { + return initialized; + } + + @Override + public void newBucketHook(int bucketOffset) + { + heapIndexUpdater.updateHeapIndexForOffset(bucketOffset, -1); + } + + @Override + public boolean canSkipAggregate(boolean bucketWasUsed, int bucketOffset) + { + if (bucketWasUsed) { + if (!sortHasNonGroupingFields) { + if (heapIndexUpdater.getHeapIndexForOffset(bucketOffset) < 0) { + return true; + } + } + } + return false; + } + + @Override + public void afterAggregateHook(int bucketOffset) + { + int heapIndex = heapIndexUpdater.getHeapIndexForOffset(bucketOffset); + if (heapIndex < 0) { + // not in the heap, add it + offsetHeap.addOffset(bucketOffset); + } else if (sortHasNonGroupingFields) { + // Since the sorting columns contain at least one aggregator, we need to remove and reinsert + // the entries after aggregating to maintain proper ordering + offsetHeap.removeAt(heapIndex); + offsetHeap.addOffset(bucketOffset); + } + } + + @Override + public void reset() + { + hashTable.reset(); + keySerde.reset(); + offsetHeap.reset(); + heapIndexUpdater.setHashTableBuffer(hashTable.getTableBuffer()); + } + + @Override + public Iterator> iterator(boolean sorted) + { + if (!initialized) { + // it's possible for iterator() to be called before initialization when + // a nested groupBy's subquery has an empty result set (see testEmptySubqueryWithLimitPushDown() + // in GroupByQueryRunnerTest) + return Iterators.>emptyIterator(); + } + + if (sortHasNonGroupingFields) { + // re-sort the heap in place, it's also an array of offsets in the totalBuffer + return makeDefaultOrderingIterator(); + } else { + return makeHeapIterator(); + } + } + + public int getLimit() + { + return limit; + } + + public static class BufferGrouperOffsetHeapIndexUpdater + { + private ByteBuffer hashTableBuffer; + private final int indexPosition; + + public BufferGrouperOffsetHeapIndexUpdater( + ByteBuffer hashTableBuffer, + int indexPosition + ) + { + this.hashTableBuffer = hashTableBuffer; + this.indexPosition = indexPosition; + } + + public void setHashTableBuffer(ByteBuffer newTableBuffer) { + hashTableBuffer = newTableBuffer; + } + + public void updateHeapIndexForOffset(int bucketOffset, int newHeapIndex) + { + hashTableBuffer.putInt(bucketOffset + indexPosition, newHeapIndex); + } + + public int getHeapIndexForOffset(int bucketOffset) + { + return hashTableBuffer.getInt(bucketOffset + indexPosition); + } + } + + private Iterator> makeDefaultOrderingIterator() + { + final int size = offsetHeap.getHeapSize(); + + final List wrappedOffsets = new AbstractList() + { + @Override + public Integer get(int index) + { + return offsetHeap.getAt(index); + } + + @Override + public Integer set(int index, Integer element) + { + final Integer oldValue = get(index); + offsetHeap.setAt(index, element); + return oldValue; + } + + @Override + public int size() + { + return size; + } + }; + + final BufferComparator comparator = keySerde.bufferComparator(); + + // Sort offsets in-place. + Collections.sort( + wrappedOffsets, + new Comparator() + { + @Override + public int compare(Integer lhs, Integer rhs) + { + final ByteBuffer curHashTableBuffer = hashTable.getTableBuffer(); + return comparator.compare( + curHashTableBuffer, + curHashTableBuffer, + lhs + HASH_SIZE, + rhs + HASH_SIZE + ); + } + } + ); + + return new Iterator>() + { + int curr = 0; + + @Override + public boolean hasNext() + { + return curr < size; + } + + @Override + public Grouper.Entry next() + { + return bucketEntryForOffset(wrappedOffsets.get(curr++)); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + + private Iterator> makeHeapIterator() + { + final int initialHeapSize = offsetHeap.getHeapSize(); + return new Iterator>() + { + int curr = 0; + + @Override + public boolean hasNext() + { + return curr < initialHeapSize; + } + + @Override + public Grouper.Entry next() + { + if (curr >= initialHeapSize) { + throw new NoSuchElementException(); + } + final int offset = offsetHeap.removeMin(); + final Grouper.Entry entry = bucketEntryForOffset(offset); + curr++; + + return entry; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + + private Comparator makeHeapComparator() + { + return new Comparator() + { + final BufferComparator bufferComparator = keySerde.bufferComparatorWithAggregators( + aggregatorFactories, + aggregatorOffsets + ); + @Override + public int compare(Integer o1, Integer o2) + { + final ByteBuffer tableBuffer = hashTable.getTableBuffer(); + return bufferComparator.compare(tableBuffer, tableBuffer, o1 + HASH_SIZE, o2 + HASH_SIZE); + } + }; + } + + + private void validateBufferCapacity( + int limit, + float maxLoadFactor, + ByteBuffer buffer, + int bucketSize + ) + { + int numBucketsNeeded = (int) Math.ceil((limit + 1) / maxLoadFactor); + int targetTableArenaSize = numBucketsNeeded * bucketSize * 2; + int heapSize = (limit + 1) * (Ints.BYTES); + int requiredSize = targetTableArenaSize + heapSize; + + if (buffer.capacity() < requiredSize) { + throw new IAE( + "Buffer capacity [%d] is too small for limit[%d] with load factor[%f], minimum bytes needed: [%d]", + buffer.capacity(), + limit, + maxLoadFactor, + requiredSize + ); + } + } + + private class AlternatingByteBufferHashTable extends ByteBufferHashTable + { + // The base buffer is split into two alternating halves, with one sub-buffer in use at a given time. + // When the current sub-buffer fills, the used bits of the other sub-buffer are cleared, entries up to the limit + // are copied from the current full sub-buffer to the new buffer, and the active buffer (referenced by tableBuffer) + // is swapped to the new buffer. + private ByteBuffer[] subHashTableBuffers; + + public AlternatingByteBufferHashTable( + float maxLoadFactor, + int initialBuckets, + int bucketSizeWithHash, + ByteBuffer totalHashTableBuffer, + int keySize, + int maxSizeForTesting + ) + { + super( + maxLoadFactor, + initialBuckets, + bucketSizeWithHash, + totalHashTableBuffer, + keySize, + maxSizeForTesting, + null + ); + + this.growthCount = 0; + + int subHashTableSize = tableArenaSize / 2; + maxBuckets = subHashTableSize / bucketSizeWithHash; + regrowthThreshold = maxSizeForBuckets(maxBuckets); + + // split the hashtable into 2 sub tables that we rotate between + ByteBuffer subHashTable1Buffer = totalHashTableBuffer.duplicate(); + subHashTable1Buffer.position(0); + subHashTable1Buffer.limit(subHashTableSize); + subHashTable1Buffer = subHashTable1Buffer.slice(); + + ByteBuffer subHashTable2Buffer = totalHashTableBuffer.duplicate(); + subHashTable2Buffer.position(subHashTableSize); + subHashTable2Buffer.limit(tableArenaSize); + subHashTable2Buffer = subHashTable2Buffer.slice(); + + subHashTableBuffers = new ByteBuffer[] {subHashTable1Buffer, subHashTable2Buffer}; + } + + @Override + public void reset() + { + size = 0; + growthCount = 0; + // clear the used bits of the first buffer + for (int i = 0; i < maxBuckets; i++) { + subHashTableBuffers[0].put(i * bucketSizeWithHash, (byte) 0); + } + tableBuffer = subHashTableBuffers[0]; + } + + @Override + public void adjustTableWhenFull() + { + int newTableIdx = (growthCount % 2 == 0) ? 1 : 0; + ByteBuffer newTableBuffer = subHashTableBuffers[newTableIdx]; + + // clear the used bits of the buffer we're swapping to + for (int i = 0; i < maxBuckets; i++) { + newTableBuffer.put(i * bucketSizeWithHash, (byte) 0); + } + + // Get the offsets of the top N buckets from the heap and copy the buckets to new table + final ByteBuffer entryBuffer = tableBuffer.duplicate(); + final ByteBuffer keyBuffer = tableBuffer.duplicate(); + + int numCopied = 0; + for (int i = 0; i < offsetHeap.getHeapSize(); i++) { + final int oldBucketOffset = offsetHeap.getAt(i); + + if (isOffsetUsed(oldBucketOffset)) { + // Read the entry from the old hash table + entryBuffer.limit(oldBucketOffset + bucketSizeWithHash); + entryBuffer.position(oldBucketOffset); + keyBuffer.limit(entryBuffer.position() + HASH_SIZE + keySize); + keyBuffer.position(entryBuffer.position() + HASH_SIZE); + + // Put the entry in the new hash table + final int keyHash = entryBuffer.getInt(entryBuffer.position()) & 0x7fffffff; + final int newBucket = findBucket(true, maxBuckets, newTableBuffer, keyBuffer, keyHash); + + if (newBucket < 0) { + throw new ISE("WTF?! Couldn't find a bucket while resizing?!"); + } + + final int newBucketOffset = newBucket * bucketSizeWithHash; + newTableBuffer.position(newBucketOffset); + newTableBuffer.put(entryBuffer); + numCopied++; + + // Update the heap with the copied bucket's new offset in the new table + offsetHeap.setAt(i, newBucketOffset); + + // relocate aggregators (see https://github.com/druid-io/druid/pull/4071) + for (int j = 0; j < aggregators.length; j++) { + aggregators[j].relocate( + oldBucketOffset + aggregatorOffsets[j], + newBucketOffset + aggregatorOffsets[j], + tableBuffer, + newTableBuffer + ); + } + } + } + + size = numCopied; + tableBuffer = newTableBuffer; + growthCount++; + } + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 211d287e7061..6ce1f85f9f97 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -29,6 +29,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Chars; +import com.google.common.primitives.Doubles; import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; @@ -47,7 +48,11 @@ import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.RowBasedColumnSelectorFactory; +import io.druid.query.groupby.orderby.DefaultLimitSpec; +import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.groupby.strategy.GroupByStrategyV2; +import io.druid.query.ordering.StringComparator; +import io.druid.query.ordering.StringComparators; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DimensionHandlerUtils; @@ -65,8 +70,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; // this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor public class RowBasedGrouperHelper @@ -95,13 +102,7 @@ public static Pair, Accumulator> crea final GroupByQueryConfig querySpecificConfig = config.withOverrides(query); final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null; - final Grouper.KeySerdeFactory keySerdeFactory = new RowBasedKeySerdeFactory( - includeTimestamp, - query.getContextSortByDimsFirst(), - query.getDimensions().size(), - querySpecificConfig.getMaxMergingDictionarySize() / (concurrencyHint == -1 ? 1 : concurrencyHint), - valueTypes - ); + final ThreadLocal columnSelectorRow = new ThreadLocal<>(); final ColumnSelectorFactory columnSelectorFactory = query.getVirtualColumns().wrap( RowBasedColumnSelectorFactory.create( @@ -109,6 +110,27 @@ public static Pair, Accumulator> crea rawInputRowSignature ) ); + + final boolean willApplyLimitPushDown = query.isApplyLimitPushDown(); + final DefaultLimitSpec limitSpec = willApplyLimitPushDown ? (DefaultLimitSpec) query.getLimitSpec() : null; + boolean sortHasNonGroupingFields = false; + if (willApplyLimitPushDown) { + sortHasNonGroupingFields = DefaultLimitSpec.sortingOrderHasNonGroupingFields( + limitSpec, + query.getDimensions() + ); + } + + final Grouper.KeySerdeFactory keySerdeFactory = new RowBasedKeySerdeFactory( + includeTimestamp, + query.getContextSortByDimsFirst(), + query.getDimensions(), + querySpecificConfig.getMaxMergingDictionarySize() / (concurrencyHint == -1 ? 1 : concurrencyHint), + valueTypes, + aggregatorFactories, + limitSpec + ); + final Grouper grouper; if (concurrencyHint == -1) { grouper = new SpillingGrouper<>( @@ -121,7 +143,9 @@ public static Pair, Accumulator> crea querySpecificConfig.getBufferGrouperInitialBuckets(), temporaryStorage, spillMapper, - true + true, + limitSpec, + sortHasNonGroupingFields ); } else { grouper = new ConcurrentGrouper<>( @@ -134,7 +158,9 @@ public static Pair, Accumulator> crea querySpecificConfig.getBufferGrouperInitialBuckets(), temporaryStorage, spillMapper, - concurrencyHint + concurrencyHint, + limitSpec, + sortHasNonGroupingFields ); } @@ -526,20 +552,6 @@ private static Supplier[] getValueSuppliersForDimensions( return inputRawSuppliers; } - @SuppressWarnings("unchecked") - private static Function[] makeValueConvertFunctions( - final Map rawInputRowSignature, - final List dimensions - ) - { - final List valueTypes = Lists.newArrayListWithCapacity(dimensions.size()); - for (DimensionSpec dimensionSpec : dimensions) { - final ValueType valueType = rawInputRowSignature.get(dimensionSpec); - valueTypes.add(valueType == null ? ValueType.STRING : valueType); - } - return makeValueConvertFunctions(valueTypes); - } - @SuppressWarnings("unchecked") private static Function[] makeValueConvertFunctions( final List valueTypes @@ -600,70 +612,207 @@ private static class RowBasedKeySerdeFactory implements Grouper.KeySerdeFactory< private final boolean sortByDimsFirst; private final int dimCount; private final long maxDictionarySize; + private final DefaultLimitSpec limitSpec; + private final List dimensions; + final AggregatorFactory[] aggregatorFactories; private final List valueTypes; RowBasedKeySerdeFactory( boolean includeTimestamp, boolean sortByDimsFirst, - int dimCount, + List dimensions, long maxDictionarySize, - List valueTypes + List valueTypes, + final AggregatorFactory[] aggregatorFactories, + DefaultLimitSpec limitSpec ) { this.includeTimestamp = includeTimestamp; this.sortByDimsFirst = sortByDimsFirst; - this.dimCount = dimCount; + this.dimensions = dimensions; + this.dimCount = dimensions.size(); this.maxDictionarySize = maxDictionarySize; + this.limitSpec = limitSpec; + this.aggregatorFactories = aggregatorFactories; this.valueTypes = valueTypes; } @Override public Grouper.KeySerde factorize() { - return new RowBasedKeySerde(includeTimestamp, sortByDimsFirst, dimCount, maxDictionarySize, valueTypes); + return new RowBasedKeySerde( + includeTimestamp, + sortByDimsFirst, + dimensions, + maxDictionarySize, + limitSpec, + valueTypes + ); } @Override - public Comparator objectComparator() + public Comparator> objectComparator(boolean forceDefaultOrder) + { + if (limitSpec != null && !forceDefaultOrder) { + return objectComparatorWithAggs(); + } + + if (includeTimestamp) { + if (sortByDimsFirst) { + return new Comparator>() + { + @Override + public int compare(Grouper.Entry entry1, Grouper.Entry entry2) + { + final int cmp = compareDimsInRows(entry1.getKey(), entry2.getKey(), 1); + if (cmp != 0) { + return cmp; + } + + return Longs.compare((long) entry1.getKey().getKey()[0], (long) entry2.getKey().getKey()[0]); + } + }; + } else { + return new Comparator>() + { + @Override + public int compare(Grouper.Entry entry1, Grouper.Entry entry2) + { + final int timeCompare = Longs.compare( + (long) entry1.getKey().getKey()[0], + (long) entry2.getKey().getKey()[0] + ); + + if (timeCompare != 0) { + return timeCompare; + } + + return compareDimsInRows(entry1.getKey(), entry2.getKey(), 1); + } + }; + } + } else { + return new Comparator>() + { + @Override + public int compare(Grouper.Entry entry1, Grouper.Entry entry2) + { + return compareDimsInRows(entry1.getKey(), entry2.getKey(), 0); + } + }; + } + } + + private Comparator> objectComparatorWithAggs() { + // use the actual sort order from the limitspec if pushing down to merge partial results correctly + final List needsReverses = Lists.newArrayList(); + final List aggFlags = Lists.newArrayList(); + final List isNumericField = Lists.newArrayList(); + final List comparators = Lists.newArrayList(); + final List fieldIndices = Lists.newArrayList(); + final Set orderByIndices = new HashSet<>(); + + for (OrderByColumnSpec orderSpec : limitSpec.getColumns()) { + final boolean needsReverse = orderSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING; + int dimIndex = OrderByColumnSpec.getDimIndexForOrderBy(orderSpec, dimensions); + if (dimIndex >= 0) { + fieldIndices.add(dimIndex); + orderByIndices.add(dimIndex); + needsReverses.add(needsReverse); + aggFlags.add(false); + final ValueType type = dimensions.get(dimIndex).getOutputType(); + isNumericField.add(type == ValueType.LONG || type == ValueType.FLOAT); + comparators.add(orderSpec.getDimensionComparator()); + } else { + int aggIndex = OrderByColumnSpec.getAggIndexForOrderBy(orderSpec, Arrays.asList(aggregatorFactories)); + if (aggIndex >= 0) { + fieldIndices.add(aggIndex); + needsReverses.add(needsReverse); + aggFlags.add(true); + final String typeName = aggregatorFactories[aggIndex].getTypeName(); + isNumericField.add(typeName.equals("long") || typeName.equals("float")); + comparators.add(orderSpec.getDimensionComparator()); + } + } + } + + for (int i = 0; i < dimCount; i++) { + if (!orderByIndices.contains(i)) { + fieldIndices.add(i); + aggFlags.add(false); + needsReverses.add(false); + final ValueType type = dimensions.get(i).getOutputType(); + isNumericField.add(type == ValueType.LONG || type == ValueType.FLOAT); + comparators.add(StringComparators.LEXICOGRAPHIC); + } + } + if (includeTimestamp) { if (sortByDimsFirst) { - return new Comparator() + return new Comparator>() { @Override - public int compare(RowBasedKey key1, RowBasedKey key2) + public int compare(Grouper.Entry entry1, Grouper.Entry entry2) { - final int cmp = compareDimsInRows(key1, key2, 1); + final int cmp = compareDimsInRowsWithAggs( + entry1, + entry2, + 1, + needsReverses, + aggFlags, + fieldIndices, + isNumericField, + comparators + ); if (cmp != 0) { return cmp; } - return Longs.compare((long) key1.getKey()[0], (long) key2.getKey()[0]); + return Longs.compare((long) entry1.getKey().getKey()[0], (long) entry2.getKey().getKey()[0]); } }; } else { - return new Comparator() + return new Comparator>() { @Override - public int compare(RowBasedKey key1, RowBasedKey key2) + public int compare(Grouper.Entry entry1, Grouper.Entry entry2) { - final int timeCompare = Longs.compare((long) key1.getKey()[0], (long) key2.getKey()[0]); + final int timeCompare = Longs.compare((long) entry1.getKey().getKey()[0], (long) entry2.getKey().getKey()[0]); if (timeCompare != 0) { return timeCompare; } - return compareDimsInRows(key1, key2, 1); + return compareDimsInRowsWithAggs( + entry1, + entry2, + 1, + needsReverses, + aggFlags, + fieldIndices, + isNumericField, + comparators + ); } }; } } else { - return new Comparator() + return new Comparator>() { @Override - public int compare(RowBasedKey key1, RowBasedKey key2) + public int compare(Grouper.Entry entry1, Grouper.Entry entry2) { - return compareDimsInRows(key1, key2, 0); + return compareDimsInRowsWithAggs( + entry1, + entry2, + 0, + needsReverses, + aggFlags, + fieldIndices, + isNumericField, + comparators + ); } }; } @@ -680,22 +829,77 @@ private static int compareDimsInRows(RowBasedKey key1, RowBasedKey key2, int dim return 0; } + + private static int compareDimsInRowsWithAggs( + Grouper.Entry entry1, + Grouper.Entry entry2, + int dimStart, + final List needsReverses, + final List aggFlags, + final List fieldIndices, + final List isNumericField, + final List comparators + ) + { + for (int i = 0; i < fieldIndices.size(); i++) { + final int fieldIndex = fieldIndices.get(i); + final boolean needsReverse = needsReverses.get(i); + final int cmp; + final Comparable lhs; + final Comparable rhs; + + if (aggFlags.get(i)) { + if (needsReverse) { + lhs = (Comparable) entry2.getValues()[fieldIndex]; + rhs = (Comparable) entry1.getValues()[fieldIndex]; + } else { + lhs = (Comparable) entry1.getValues()[fieldIndex]; + rhs = (Comparable) entry2.getValues()[fieldIndex]; + } + } else { + if (needsReverse) { + lhs = (Comparable) entry2.getKey().getKey()[fieldIndex + dimStart]; + rhs = (Comparable) entry1.getKey().getKey()[fieldIndex + dimStart]; + } else { + lhs = (Comparable) entry1.getKey().getKey()[fieldIndex + dimStart]; + rhs = (Comparable) entry2.getKey().getKey()[fieldIndex + dimStart]; + } + } + + final StringComparator comparator = comparators.get(i); + + if (isNumericField.get(i) && comparator == StringComparators.NUMERIC) { + // use natural comparison + cmp = lhs.compareTo(rhs); + } else { + cmp = comparator.compare(lhs.toString(), rhs.toString()); + } + + if (cmp != 0) { + return cmp; + } + } + + return 0; + } } - private static class RowBasedKeySerde implements Grouper.KeySerde + private static class RowBasedKeySerde implements Grouper.KeySerde { // Entry in dictionary, node pointer in reverseDictionary, hash + k/v/next pointer in reverseDictionary nodes private static final int ROUGH_OVERHEAD_PER_DICTIONARY_ENTRY = Longs.BYTES * 5 + Ints.BYTES; private final boolean includeTimestamp; private final boolean sortByDimsFirst; + private final List dimensions; private final int dimCount; private final int keySize; private final ByteBuffer keyBuffer; private final List dictionary = Lists.newArrayList(); private final Map reverseDictionary = Maps.newHashMap(); - private final List valueTypes; private final List serdeHelpers; + private final DefaultLimitSpec limitSpec; + private final List valueTypes; // Size limiting for the dictionary, in (roughly estimated) bytes. private final long maxDictionarySize; @@ -707,16 +911,19 @@ private static class RowBasedKeySerde implements Grouper.KeySerde RowBasedKeySerde( final boolean includeTimestamp, final boolean sortByDimsFirst, - final int dimCount, + final List dimensions, final long maxDictionarySize, + final DefaultLimitSpec limitSpec, final List valueTypes ) { this.includeTimestamp = includeTimestamp; this.sortByDimsFirst = sortByDimsFirst; - this.dimCount = dimCount; + this.dimensions = dimensions; + this.dimCount = dimensions.size(); this.maxDictionarySize = maxDictionarySize; this.valueTypes = valueTypes; + this.limitSpec = limitSpec; this.serdeHelpers = makeSerdeHelpers(); this.keySize = (includeTimestamp ? Longs.BYTES : 0) + getTotalKeySize(); this.keyBuffer = ByteBuffer.allocate(keySize); @@ -783,7 +990,7 @@ public RowBasedKey fromByteBuffer(ByteBuffer buffer, int position) } @Override - public Grouper.KeyComparator bufferComparator() + public Grouper.BufferComparator bufferComparator() { if (sortableIds == null) { Map sortedMap = Maps.newTreeMap(); @@ -799,7 +1006,7 @@ public Grouper.KeyComparator bufferComparator() if (includeTimestamp) { if (sortByDimsFirst) { - return new Grouper.KeyComparator() + return new Grouper.BufferComparator() { @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) @@ -821,7 +1028,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } }; } else { - return new Grouper.KeyComparator() + return new Grouper.BufferComparator() { @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) @@ -845,7 +1052,7 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, }; } } else { - return new Grouper.KeyComparator() + return new Grouper.BufferComparator() { @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) @@ -869,6 +1076,158 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } } + @Override + public Grouper.BufferComparator bufferComparatorWithAggregators( + AggregatorFactory[] aggregatorFactories, + int[] aggregatorOffsets + ) + { + final List adjustedSerdeHelpers; + final List needsReverses = Lists.newArrayList(); + List orderByHelpers = new ArrayList<>(); + List otherDimHelpers = new ArrayList<>(); + Set orderByIndices = new HashSet<>(); + + int aggCount = 0; + boolean needsReverse; + for (OrderByColumnSpec orderSpec : limitSpec.getColumns()) { + needsReverse = orderSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING; + int dimIndex = OrderByColumnSpec.getDimIndexForOrderBy(orderSpec, dimensions); + if (dimIndex >= 0) { + RowBasedKeySerdeHelper serdeHelper = serdeHelpers.get(dimIndex); + orderByHelpers.add(serdeHelper); + orderByIndices.add(dimIndex); + needsReverses.add(needsReverse); + } else { + int aggIndex = OrderByColumnSpec.getAggIndexForOrderBy(orderSpec, Arrays.asList(aggregatorFactories)); + if (aggIndex >= 0) { + final RowBasedKeySerdeHelper serdeHelper; + final StringComparator cmp = orderSpec.getDimensionComparator(); + final boolean cmpIsNumeric = cmp == StringComparators.NUMERIC; + final String typeName = aggregatorFactories[aggIndex].getTypeName(); + final int aggOffset = aggregatorOffsets[aggIndex] - Ints.BYTES; + + aggCount++; + + if (typeName.equals("long")) { + if (cmpIsNumeric) { + serdeHelper = new LongRowBasedKeySerdeHelper(aggOffset); + } else { + serdeHelper = new LimitPushDownLongRowBasedKeySerdeHelper(aggOffset, cmp); + } + } else if (typeName.equals("float")) { + // called "float", but the aggs really return doubles + if (cmpIsNumeric) { + serdeHelper = new DoubleRowBasedKeySerdeHelper(aggOffset); + } else { + serdeHelper = new LimitPushDownDoubleRowBasedKeySerdeHelper(aggOffset, cmp); + } + } else { + throw new IAE("Cannot order by a non-numeric aggregator[%s]", orderSpec); + } + + orderByHelpers.add(serdeHelper); + needsReverses.add(needsReverse); + } + } + } + + for (int i = 0; i < dimCount; i++) { + if (!orderByIndices.contains(i)) { + otherDimHelpers.add(serdeHelpers.get(i)); + needsReverses.add(false); // default to Ascending order if dim is not in an orderby spec + } + } + + adjustedSerdeHelpers = orderByHelpers; + adjustedSerdeHelpers.addAll(otherDimHelpers); + + final int fieldCount = dimCount + aggCount; + + if (includeTimestamp) { + if (sortByDimsFirst) { + return new Grouper.BufferComparator() + { + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + final int cmp = compareDimsInBuffersForNullFudgeTimestampForPushDown( + adjustedSerdeHelpers, + needsReverses, + fieldCount, + lhsBuffer, + rhsBuffer, + lhsPosition, + rhsPosition + ); + if (cmp != 0) { + return cmp; + } + + return Longs.compare(lhsBuffer.getLong(lhsPosition), rhsBuffer.getLong(rhsPosition)); + } + }; + } else { + return new Grouper.BufferComparator() + { + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + final int timeCompare = Longs.compare(lhsBuffer.getLong(lhsPosition), rhsBuffer.getLong(rhsPosition)); + + if (timeCompare != 0) { + return timeCompare; + } + + int cmp = compareDimsInBuffersForNullFudgeTimestampForPushDown( + adjustedSerdeHelpers, + needsReverses, + fieldCount, + lhsBuffer, + rhsBuffer, + lhsPosition, + rhsPosition + ); + + return cmp; + } + }; + } + } else { + return new Grouper.BufferComparator() + { + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + for (int i = 0; i < fieldCount; i++) { + final int cmp; + if (needsReverses.get(i)) { + cmp = adjustedSerdeHelpers.get(i).compare( + rhsBuffer, + lhsBuffer, + rhsPosition, + lhsPosition + ); + } else { + cmp = adjustedSerdeHelpers.get(i).compare( + lhsBuffer, + rhsBuffer, + lhsPosition, + rhsPosition + ); + } + + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + }; + } + } + private static int compareDimsInBuffersForNullFudgeTimestamp( List serdeHelpers, int[] sortableIds, @@ -894,6 +1253,41 @@ private static int compareDimsInBuffersForNullFudgeTimestamp( return 0; } + private static int compareDimsInBuffersForNullFudgeTimestampForPushDown( + List serdeHelpers, + List needsReverses, + int dimCount, + ByteBuffer lhsBuffer, + ByteBuffer rhsBuffer, + int lhsPosition, + int rhsPosition + ) + { + for (int i = 0; i < dimCount; i++) { + final int cmp; + if (needsReverses.get(i)) { + cmp = serdeHelpers.get(i).compare( + rhsBuffer, + lhsBuffer, + rhsPosition + Longs.BYTES, + lhsPosition + Longs.BYTES + ); + } else { + cmp = serdeHelpers.get(i).compare( + lhsBuffer, + rhsBuffer, + lhsPosition + Longs.BYTES, + rhsPosition + Longs.BYTES + ); + } + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + @Override public void reset() { @@ -939,6 +1333,10 @@ private int getTotalKeySize() private List makeSerdeHelpers() { + if (limitSpec != null) { + return makeSerdeHelpersForLimitPushDown(); + } + List helpers = new ArrayList<>(); int keyBufferPosition = 0; for (ValueType valType : valueTypes) { @@ -962,6 +1360,48 @@ private List makeSerdeHelpers() return helpers; } + private List makeSerdeHelpersForLimitPushDown() + { + List helpers = new ArrayList<>(); + int keyBufferPosition = 0; + + for (int i = 0; i < valueTypes.size(); i++) { + final ValueType valType = valueTypes.get(i); + final String dimName = dimensions.get(i).getOutputName(); + StringComparator cmp = DefaultLimitSpec.getComparatorForDimName(limitSpec, dimName); + final boolean cmpIsNumeric = cmp == StringComparators.NUMERIC; + + RowBasedKeySerdeHelper helper; + switch (valType) { + case STRING: + if (cmp == null) { + cmp = StringComparators.LEXICOGRAPHIC; + } + helper = new LimitPushDownStringRowBasedKeySerdeHelper(keyBufferPosition, cmp); + break; + case LONG: + if (cmp == null || cmpIsNumeric) { + helper = new LongRowBasedKeySerdeHelper(keyBufferPosition); + } else { + helper = new LimitPushDownLongRowBasedKeySerdeHelper(keyBufferPosition, cmp); + } + break; + case FLOAT: + if (cmp == null || cmpIsNumeric) { + helper = new FloatRowBasedKeySerdeHelper(keyBufferPosition); + } else { + helper = new LimitPushDownFloatRowBasedKeySerdeHelper(keyBufferPosition, cmp); + } + break; + default: + throw new IAE("invalid type: %s", valType); + } + keyBufferPosition += helper.getKeyBufferValueSize(); + helpers.add(helper); + } + return helpers; + } + private interface RowBasedKeySerdeHelper { /** @@ -1053,6 +1493,25 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } } + private class LimitPushDownStringRowBasedKeySerdeHelper extends StringRowBasedKeySerdeHelper + { + final StringComparator cmp; + + public LimitPushDownStringRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) + { + super(keyBufferPosition); + this.cmp = cmp; + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + String lhsStr = dictionary.get(lhsBuffer.getInt(lhsPosition + keyBufferPosition)); + String rhsStr = dictionary.get(rhsBuffer.getInt(rhsPosition + keyBufferPosition)); + return cmp.compare(lhsStr, rhsStr); + } + } + private class LongRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { final int keyBufferPosition; @@ -1091,6 +1550,26 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, } } + private class LimitPushDownLongRowBasedKeySerdeHelper extends LongRowBasedKeySerdeHelper + { + final StringComparator cmp; + + public LimitPushDownLongRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) + { + super(keyBufferPosition); + this.cmp = cmp; + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + long lhs = lhsBuffer.getLong(lhsPosition + keyBufferPosition); + long rhs = rhsBuffer.getLong(rhsPosition + keyBufferPosition); + + return cmp.compare(String.valueOf(lhs), String.valueOf(rhs)); + } + } + private class FloatRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper { final int keyBufferPosition; @@ -1128,5 +1607,81 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, ); } } + + private class LimitPushDownFloatRowBasedKeySerdeHelper extends FloatRowBasedKeySerdeHelper + { + final StringComparator cmp; + + public LimitPushDownFloatRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) + { + super(keyBufferPosition); + this.cmp = cmp; + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + float lhs = lhsBuffer.getFloat(lhsPosition + keyBufferPosition); + float rhs = rhsBuffer.getFloat(rhsPosition + keyBufferPosition); + return cmp.compare(String.valueOf(lhs), String.valueOf(rhs)); + } + } + + private class DoubleRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper + { + final int keyBufferPosition; + + public DoubleRowBasedKeySerdeHelper(int keyBufferPosition) + { + this.keyBufferPosition = keyBufferPosition; + } + + @Override + public int getKeyBufferValueSize() + { + return Doubles.BYTES; + } + + @Override + public boolean putToKeyBuffer(RowBasedKey key, int idx) + { + keyBuffer.putDouble((Double) key.getKey()[idx]); + return true; + } + + @Override + public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues) + { + dimValues[dimValIdx] = buffer.getDouble(initialOffset + keyBufferPosition); + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return Double.compare( + lhsBuffer.getDouble(lhsPosition + keyBufferPosition), + rhsBuffer.getDouble(rhsPosition + keyBufferPosition) + ); + } + } + + private class LimitPushDownDoubleRowBasedKeySerdeHelper extends DoubleRowBasedKeySerdeHelper + { + final StringComparator cmp; + + public LimitPushDownDoubleRowBasedKeySerdeHelper(int keyBufferPosition, StringComparator cmp) + { + super(keyBufferPosition); + this.cmp = cmp; + } + + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + double lhs = lhsBuffer.getDouble(lhsPosition + keyBufferPosition); + double rhs = rhsBuffer.getDouble(rhsPosition + keyBufferPosition); + return cmp.compare(String.valueOf(lhs), String.valueOf(rhs)); + } + } } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java index 21490173cbac..41d8bdba178b 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java @@ -30,6 +30,7 @@ import io.druid.java.util.common.guava.CloseQuietly; import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.segment.ColumnSelectorFactory; import net.jpountz.lz4.LZ4BlockInputStream; import net.jpountz.lz4.LZ4BlockOutputStream; @@ -51,19 +52,20 @@ */ public class SpillingGrouper implements Grouper { + private final Grouper grouper; private static final AggregateResult DISK_FULL = AggregateResult.failure( "Not enough disk space to execute this query. Try raising druid.query.groupBy.maxOnDiskStorage." ); - - private final BufferGrouper grouper; private final KeySerde keySerde; private final LimitedTemporaryStorage temporaryStorage; private final ObjectMapper spillMapper; private final AggregatorFactory[] aggregatorFactories; - private final Comparator keyObjComparator; + private final Comparator> keyObjComparator; + private final Comparator> defaultOrderKeyObjComparator; private final List files = Lists.newArrayList(); private final List closeables = Lists.newArrayList(); + private final boolean sortHasNonGroupingFields; private boolean spillingAllowed = false; @@ -77,24 +79,42 @@ public SpillingGrouper( final int bufferGrouperInitialBuckets, final LimitedTemporaryStorage temporaryStorage, final ObjectMapper spillMapper, - final boolean spillingAllowed + final boolean spillingAllowed, + final DefaultLimitSpec limitSpec, + final boolean sortHasNonGroupingFields ) { this.keySerde = keySerdeFactory.factorize(); - this.keyObjComparator = keySerdeFactory.objectComparator(); - this.grouper = new BufferGrouper<>( - bufferSupplier, - keySerde, - columnSelectorFactory, - aggregatorFactories, - bufferGrouperMaxSize, - bufferGrouperMaxLoadFactor, - bufferGrouperInitialBuckets - ); + this.keyObjComparator = keySerdeFactory.objectComparator(false); + this.defaultOrderKeyObjComparator = keySerdeFactory.objectComparator(true); + if (limitSpec != null) { + this.grouper = new LimitedBufferGrouper<>( + bufferSupplier, + keySerde, + columnSelectorFactory, + aggregatorFactories, + bufferGrouperMaxSize, + bufferGrouperMaxLoadFactor, + bufferGrouperInitialBuckets, + limitSpec.getLimit(), + sortHasNonGroupingFields + ); + } else { + this.grouper = new BufferGrouper<>( + bufferSupplier, + keySerde, + columnSelectorFactory, + aggregatorFactories, + bufferGrouperMaxSize, + bufferGrouperMaxLoadFactor, + bufferGrouperInitialBuckets + ); + } this.aggregatorFactories = aggregatorFactories; this.temporaryStorage = temporaryStorage; this.spillMapper = spillMapper; this.spillingAllowed = spillingAllowed; + this.sortHasNonGroupingFields = sortHasNonGroupingFields; } @Override @@ -191,7 +211,11 @@ public Entry apply(Entry entry) closeables.add(fileIterator); } - return Groupers.mergeIterators(iterators, sorted ? keyObjComparator : null); + if (sortHasNonGroupingFields) { + return Groupers.mergeIterators(iterators, defaultOrderKeyObjComparator); + } else { + return Groupers.mergeIterators(iterators, sorted ? keyObjComparator : null); + } } private void spill() throws IOException diff --git a/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java index 2279cac984ff..6528cfc7bbcf 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java @@ -26,17 +26,9 @@ */ public class AlwaysHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x0; - @Override public boolean eval(Row row) { return true; } - - @Override - public byte[] getCacheKey() - { - return new byte[]{CACHE_KEY}; - } } diff --git a/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java index a31d3537ade7..4a5f962bcdca 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java @@ -25,7 +25,6 @@ import io.druid.data.input.Row; import io.druid.segment.column.ValueType; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -34,9 +33,7 @@ */ public class AndHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x2; - - private List havingSpecs; + private final List havingSpecs; @JsonCreator public AndHavingSpec(@JsonProperty("havingSpecs") List havingSpecs) @@ -70,25 +67,6 @@ public boolean eval(Row row) return true; } - @Override - public byte[] getCacheKey() - { - final byte[][] havingBytes = new byte[havingSpecs.size()][]; - int havingBytesSize = 0; - int index = 0; - for (HavingSpec havingSpec : havingSpecs) { - havingBytes[index] = havingSpec.getCacheKey(); - havingBytesSize += havingBytes[index].length; - ++index; - } - - ByteBuffer buffer = ByteBuffer.allocate(1 + havingBytesSize).put(CACHE_KEY); - for (byte[] havingByte : havingBytes) { - buffer.put(havingByte); - } - return buffer.array(); - } - @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/groupby/having/DimFilterHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/DimFilterHavingSpec.java index 12abeb35099c..6051b96aac05 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/DimFilterHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/DimFilterHavingSpec.java @@ -29,15 +29,13 @@ import io.druid.query.groupby.RowBasedColumnSelectorFactory; import io.druid.segment.column.ValueType; -import java.nio.ByteBuffer; import java.util.Map; public class DimFilterHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = (byte) 0x9; - private final DimFilter dimFilter; private final SettableSupplier rowSupplier; + private ValueMatcher valueMatcher; private int evalCount; @@ -77,16 +75,6 @@ public boolean eval(final Row row) return retVal; } - @Override - public byte[] getCacheKey() - { - final byte[] filterBytes = dimFilter.getCacheKey(); - return ByteBuffer.allocate(1 + filterBytes.length) - .put(CACHE_KEY) - .put(filterBytes) - .array(); - } - @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/groupby/having/DimensionSelectorHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/DimensionSelectorHavingSpec.java index 5aeade63cd93..c4302931812e 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/DimensionSelectorHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/DimensionSelectorHavingSpec.java @@ -24,17 +24,14 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import io.druid.data.input.Row; -import io.druid.java.util.common.StringUtils; import io.druid.query.extraction.ExtractionFn; import io.druid.query.extraction.IdentityExtractionFn; -import java.nio.ByteBuffer; import java.util.List; +import java.util.Objects; public class DimensionSelectorHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = (byte) 0x8; - private static final byte STRING_SEPARATOR = (byte) 0xFF; private final String dimension; private final String value; private final ExtractionFn extractionFn; @@ -69,6 +66,7 @@ public ExtractionFn getExtractionFn() return extractionFn; } + @Override public boolean eval(Row row) { List dimRowValList = row.getDimension(dimension); @@ -89,24 +87,8 @@ public boolean eval(Row row) return false; } - public byte[] getCacheKey() - { - byte[] dimBytes = StringUtils.toUtf8(dimension); - byte[] valBytes = StringUtils.toUtf8(value); - byte [] extractionFnBytes = this.getExtractionFn().getCacheKey(); - - return ByteBuffer.allocate(3 + dimBytes.length + valBytes.length + extractionFnBytes.length) - .put(CACHE_KEY) - .put(dimBytes) - .put(STRING_SEPARATOR) - .put(valBytes) - .put(STRING_SEPARATOR) - .put(extractionFnBytes) - .array(); - } - @Override - public boolean equals(Object o) + public boolean equals(final Object o) { if (this == o) { return true; @@ -114,44 +96,25 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - - DimensionSelectorHavingSpec that = (DimensionSelectorHavingSpec) o; - boolean valEquals = false; - boolean dimEquals = false; - - if (value != null && that.value != null) { - valEquals = value.equals(that.value); - } else if (value == null && that.value == null) { - valEquals = true; - } - - if (dimension != null && that.dimension != null) { - dimEquals = dimension.equals(that.dimension); - } else if (dimension == null && that.dimension == null) { - dimEquals = true; - } - - return (valEquals && dimEquals && extractionFn.equals(that.extractionFn)); + final DimensionSelectorHavingSpec that = (DimensionSelectorHavingSpec) o; + return Objects.equals(dimension, that.dimension) && + Objects.equals(value, that.value) && + Objects.equals(extractionFn, that.extractionFn); } @Override public int hashCode() { - int result = dimension != null ? dimension.hashCode() : 0; - result = 31 * result + (value != null ? value.hashCode() : 0); - return result; + return Objects.hash(dimension, value, extractionFn); } - + @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("DimensionSelectorHavingSpec"); - sb.append("{dimension='").append(dimension).append('\''); - sb.append(", value='").append(value); - sb.append("', extractionFunction='").append(getExtractionFn()); - sb.append("'}"); - return sb.toString(); + return "DimensionSelectorHavingSpec{" + + "dimension='" + dimension + '\'' + + ", value='" + value + '\'' + + ", extractionFn=" + extractionFn + + '}'; } - } diff --git a/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java index 6c2597726168..89698285b802 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java @@ -21,12 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.primitives.Bytes; import io.druid.data.input.Row; -import io.druid.java.util.common.StringUtils; - -import java.nio.ByteBuffer; -import java.util.Arrays; /** * The "=" operator in a "having" clause. This is similar to SQL's "having aggregation = value", @@ -34,10 +29,8 @@ */ public class EqualToHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x3; - - private String aggregationName; - private Number value; + private final String aggregationName; + private final Number value; @JsonCreator public EqualToHavingSpec( @@ -67,18 +60,6 @@ public boolean eval(Row row) return HavingSpecMetricComparator.compare(row, aggregationName, value) == 0; } - @Override - public byte[] getCacheKey() - { - final byte[] aggBytes = StringUtils.toUtf8(aggregationName); - final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); - return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) - .put(CACHE_KEY) - .put(aggBytes) - .put(valBytes) - .array(); - } - /** * This method treats internal value as double mainly for ease of test. */ diff --git a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java index 27fec036c742..d75333b74fae 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java @@ -21,12 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.primitives.Bytes; import io.druid.data.input.Row; -import io.druid.java.util.common.StringUtils; - -import java.nio.ByteBuffer; -import java.util.Arrays; /** * The ">" operator in a "having" clause. This is similar to SQL's "having aggregation > value", @@ -34,10 +29,8 @@ */ public class GreaterThanHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x4; - - private String aggregationName; - private Number value; + private final String aggregationName; + private final Number value; @JsonCreator public GreaterThanHavingSpec( @@ -67,18 +60,6 @@ public boolean eval(Row row) return HavingSpecMetricComparator.compare(row, aggregationName, value) > 0; } - @Override - public byte[] getCacheKey() - { - final byte[] aggBytes = StringUtils.toUtf8(aggregationName); - final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); - return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) - .put(CACHE_KEY) - .put(aggBytes) - .put(valBytes) - .array(); - } - /** * This method treats internal value as double mainly for ease of test. */ diff --git a/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java index 7efe7784fab8..d3bc31c9989b 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.data.input.Row; -import io.druid.java.util.common.Cacheable; import io.druid.segment.column.ValueType; import java.util.Map; @@ -44,7 +43,7 @@ @JsonSubTypes.Type(name = "always", value = AlwaysHavingSpec.class), @JsonSubTypes.Type(name = "filter", value = DimFilterHavingSpec.class) }) -public interface HavingSpec extends Cacheable +public interface HavingSpec { // Atoms for easy combination, but for now they are mostly useful // for testing. diff --git a/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java index 580bba13a0f8..a60d5adc123e 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java @@ -20,12 +20,7 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.primitives.Bytes; import io.druid.data.input.Row; -import io.druid.java.util.common.StringUtils; - -import java.nio.ByteBuffer; -import java.util.Arrays; /** * The "<" operator in a "having" clause. This is similar to SQL's "having aggregation < value", @@ -33,10 +28,8 @@ */ public class LessThanHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x5; - - private String aggregationName; - private Number value; + private final String aggregationName; + private final Number value; public LessThanHavingSpec( @JsonProperty("aggregation") String aggName, @@ -65,18 +58,6 @@ public boolean eval(Row row) return HavingSpecMetricComparator.compare(row, aggregationName, value) < 0; } - @Override - public byte[] getCacheKey() - { - final byte[] aggBytes = StringUtils.toUtf8(aggregationName); - final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); - return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) - .put(CACHE_KEY) - .put(aggBytes) - .put(valBytes) - .array(); - } - /** * This method treats internal value as double mainly for ease of test. */ diff --git a/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java index 6098b73359da..a609c48ffe13 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java @@ -26,17 +26,9 @@ */ public class NeverHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x1; - @Override public boolean eval(Row row) { return false; } - - @Override - public byte[] getCacheKey() - { - return new byte[]{CACHE_KEY}; - } } diff --git a/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java index 72568e02d293..b049fd668cec 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java @@ -24,7 +24,6 @@ import io.druid.data.input.Row; import io.druid.segment.column.ValueType; -import java.nio.ByteBuffer; import java.util.Map; /** @@ -32,9 +31,7 @@ */ public class NotHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x6; - - private HavingSpec havingSpec; + private final HavingSpec havingSpec; @JsonCreator public NotHavingSpec(@JsonProperty("havingSpec") HavingSpec havingSpec) @@ -60,15 +57,6 @@ public boolean eval(Row row) return !havingSpec.eval(row); } - @Override - public byte[] getCacheKey() - { - return ByteBuffer.allocate(1 + havingSpec.getCacheKey().length) - .put(CACHE_KEY) - .put(havingSpec.getCacheKey()) - .array(); - } - @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java index eed28e3023d2..67eb4edd3daa 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java @@ -25,7 +25,6 @@ import io.druid.data.input.Row; import io.druid.segment.column.ValueType; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -34,9 +33,7 @@ */ public class OrHavingSpec extends BaseHavingSpec { - private static final byte CACHE_KEY = 0x7; - - private List havingSpecs; + private final List havingSpecs; @JsonCreator public OrHavingSpec(@JsonProperty("havingSpecs") List havingSpecs) @@ -70,25 +67,6 @@ public boolean eval(Row row) return false; } - @Override - public byte[] getCacheKey() - { - final byte[][] havingBytes = new byte[havingSpecs.size()][]; - int havingBytesSize = 0; - int index = 0; - for (HavingSpec havingSpec : havingSpecs) { - havingBytes[index] = havingSpec.getCacheKey(); - havingBytesSize += havingBytes[index].length; - ++index; - } - - ByteBuffer buffer = ByteBuffer.allocate(1 + havingBytesSize).put(CACHE_KEY); - for (byte[] havingByte : havingBytes) { - buffer.put(havingByte); - } - return buffer.array(); - } - @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index da1223f6594d..fb1c1d433e14 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -57,6 +57,34 @@ public class DefaultLimitSpec implements LimitSpec private final List columns; private final int limit; + /** + * Check if a limitSpec has columns in the sorting order that are not part of the grouping fields represented + * by `dimensions`. + * + * @param limitSpec LimitSpec, assumed to be non-null + * @param dimensions Grouping fields for a groupBy query + * @return True if limitSpec has sorting columns not contained in dimensions + */ + public static boolean sortingOrderHasNonGroupingFields(DefaultLimitSpec limitSpec, List dimensions) + { + for (OrderByColumnSpec orderSpec : limitSpec.getColumns()) { + int dimIndex = OrderByColumnSpec.getDimIndexForOrderBy(orderSpec, dimensions); + if (dimIndex < 0) { + return true; + } + } + return false; + } + + public static StringComparator getComparatorForDimName(DefaultLimitSpec limitSpec, String dimName) { + final OrderByColumnSpec orderBy = OrderByColumnSpec.getOrderByForDimName(limitSpec.getColumns(), dimName); + if (orderBy == null) { + return null; + } + + return orderBy.getDimensionComparator(); + } + @JsonCreator public DefaultLimitSpec( @JsonProperty("columns") List columns, @@ -208,9 +236,8 @@ public int compare(Row left, Row right) throw new ISE("Unknown column in order clause[%s]", columnSpec); } - switch (columnSpec.getDirection()) { - case DESCENDING: - nextOrdering = nextOrdering.reverse(); + if (columnSpec.getDirection() == OrderByColumnSpec.Direction.DESCENDING) { + nextOrdering = nextOrdering.reverse(); } ordering = ordering.compound(nextOrdering); diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java index 10bc732a883d..c4a5d1c4665c 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java @@ -27,6 +27,9 @@ import com.google.common.collect.Lists; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.ordering.StringComparator; import io.druid.query.ordering.StringComparators; @@ -150,6 +153,49 @@ public OrderByColumnSpec apply(@Nullable String input) ); } + public static OrderByColumnSpec getOrderByForDimName(List orderBys, String dimName) + { + for (OrderByColumnSpec orderBy : orderBys) { + if (orderBy.dimension.equals(dimName)) { + return orderBy; + } + } + return null; + } + + public static int getDimIndexForOrderBy(OrderByColumnSpec orderSpec, List dimensions) { + int i = 0; + for (DimensionSpec dimSpec : dimensions) { + if (orderSpec.getDimension().equals((dimSpec.getOutputName()))) { + return i; + } + i++; + } + return -1; + } + + public static int getAggIndexForOrderBy(OrderByColumnSpec orderSpec, List aggregatorFactories) { + int i = 0; + for (AggregatorFactory agg : aggregatorFactories) { + if (orderSpec.getDimension().equals((agg.getName()))) { + return i; + } + i++; + } + return -1; + } + + public static int getPostAggIndexForOrderBy(OrderByColumnSpec orderSpec, List postAggs) { + int i = 0; + for (PostAggregator postAgg : postAggs) { + if (orderSpec.getDimension().equals((postAgg.getName()))) { + return i; + } + i++; + } + return -1; + } + public OrderByColumnSpec( String dimension, Direction direction diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java index 4ba495356ba3..0cf98c313503 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java @@ -37,6 +37,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.GroupByMergedQueryRunner; import io.druid.query.IntervalChunkingQueryRunnerDecorator; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryWatcher; import io.druid.query.aggregation.AggregatorFactory; @@ -120,25 +121,27 @@ public Sequence mergeResults( configSupplier.get(), bufferPool, baseRunner.run( - new GroupByQuery.Builder(query) - // Don't do post aggs until the end of this method. - .setPostAggregatorSpecs(ImmutableList.of()) - // Don't do "having" clause until the end of this method. - .setHavingSpec(null) - .setLimitSpec(NoopLimitSpec.instance()) - .overrideContext( - ImmutableMap.of( - "finalize", false, - //setting sort to false avoids unnecessary sorting while merging results. we only need to sort - //in the end when returning results to user. (note this is only respected by groupBy v1) - GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false, - //no merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would - //return merged results. (note this is only respected by groupBy v1) - GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, false, - GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1 + QueryPlus.wrap( + new GroupByQuery.Builder(query) + // Don't do post aggs until the end of this method. + .setPostAggregatorSpecs(ImmutableList.of()) + // Don't do "having" clause until the end of this method. + .setHavingSpec(null) + .setLimitSpec(NoopLimitSpec.instance()) + .overrideContext( + ImmutableMap.of( + "finalize", false, + //set sort to false avoids unnecessary sorting while merging results. we only need to sort + //in the end when returning results to user. (note this is only respected by groupBy v1) + GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false, + //no merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would + //return merged results. (note this is only respected by groupBy v1) + GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, false, + GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1 + ) ) - ) - .build(), + .build() + ), responseContext ), true diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java index e13b4e7c6fb5..ff9a43032973 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -48,6 +47,7 @@ import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryWatcher; import io.druid.query.ResourceLimitExceededException; @@ -61,7 +61,6 @@ import io.druid.query.groupby.epinephelinae.GroupByMergingQueryRunnerV2; import io.druid.query.groupby.epinephelinae.GroupByQueryEngineV2; import io.druid.query.groupby.epinephelinae.GroupByRowProcessor; -import io.druid.query.groupby.orderby.NoopLimitSpec; import io.druid.query.groupby.resource.GroupByQueryResource; import io.druid.segment.StorageAdapter; import org.joda.time.DateTime; @@ -211,7 +210,6 @@ public Sequence mergeResults( { // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may // involve materialization) - final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner(baseRunner) { @Override @@ -230,58 +228,71 @@ protected BinaryFn createMergeFn(Query queryParam) // Fudge timestamp, maybe. final DateTime fudgeTimestamp = getUniversalTimestamp(query); - return query.postProcess( - Sequences.map( - mergingQueryRunner.run( - new GroupByQuery.Builder(query) - // Don't do post aggs until the end of this method. - .setPostAggregatorSpecs(ImmutableList.of()) - // Don't do "having" clause until the end of this method. - .setHavingSpec(null) - .setLimitSpec(NoopLimitSpec.instance()) - .overrideContext( - ImmutableMap.of( - "finalize", false, - GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V2, - CTX_KEY_FUDGE_TIMESTAMP, fudgeTimestamp == null ? "" : String.valueOf(fudgeTimestamp.getMillis()), - CTX_KEY_OUTERMOST, false - ) - ) - .build(), - responseContext - ), - new Function() - { - @Override - public Row apply(final Row row) - { - // Apply postAggregators and fudgeTimestamp if present and if this is the outermost mergeResults. + final GroupByQuery newQuery = new GroupByQuery( + query.getDataSource(), + query.getQuerySegmentSpec(), + query.getVirtualColumns(), + query.getDimFilter(), + query.getGranularity(), + query.getDimensions(), + query.getAggregatorSpecs(), + query.getPostAggregatorSpecs(), + // Don't do "having" clause until the end of this method. + null, + query.getLimitSpec(), + query.getContext() + ).withOverriddenContext( + ImmutableMap.of( + "finalize", false, + GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V2, + CTX_KEY_FUDGE_TIMESTAMP, fudgeTimestamp == null ? "" : String.valueOf(fudgeTimestamp.getMillis()), + CTX_KEY_OUTERMOST, false + ) + ); - if (!query.getContextBoolean(CTX_KEY_OUTERMOST, true)) { - return row; - } + Sequence rowSequence = Sequences.map( + mergingQueryRunner.run( + QueryPlus.wrap(newQuery), + responseContext + ), + new Function() + { + @Override + public Row apply(final Row row) + { + // Apply postAggregators and fudgeTimestamp if present and if this is the outermost mergeResults. - if (query.getPostAggregatorSpecs().isEmpty() && fudgeTimestamp == null) { - return row; - } + if (!query.getContextBoolean(CTX_KEY_OUTERMOST, true)) { + return row; + } - final Map newMap; + if (query.getPostAggregatorSpecs().isEmpty() && fudgeTimestamp == null) { + return row; + } - if (query.getPostAggregatorSpecs().isEmpty()) { - newMap = ((MapBasedRow) row).getEvent(); - } else { - newMap = Maps.newLinkedHashMap(((MapBasedRow) row).getEvent()); + final Map newMap; - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - newMap.put(postAggregator.getName(), postAggregator.compute(newMap)); - } - } + if (query.getPostAggregatorSpecs().isEmpty()) { + newMap = ((MapBasedRow) row).getEvent(); + } else { + newMap = Maps.newLinkedHashMap(((MapBasedRow) row).getEvent()); - return new MapBasedRow(fudgeTimestamp != null ? fudgeTimestamp : row.getTimestamp(), newMap); + for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + newMap.put(postAggregator.getName(), postAggregator.compute(newMap)); } } - ) + + return new MapBasedRow(fudgeTimestamp != null ? fudgeTimestamp : row.getTimestamp(), newMap); + } + } ); + + // Don't apply limit here for inner results, that will be pushed down to the BufferGrouper + if (query.getContextBoolean(CTX_KEY_OUTERMOST, true)) { + return query.postProcess(rowSequence); + } else { + return rowSequence; + } } @Override @@ -304,7 +315,7 @@ public Sequence processSubqueryResult( return mergeResults(new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return results; } diff --git a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java index 46158ab177e4..88e3578f6824 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupReferencesManager.java @@ -163,6 +163,7 @@ void handlePendingNotices() return; } + @SuppressWarnings("ArgumentParameterSwap") LookupUpdateState swappedState = atomicallyUpdateStateRef( oldState -> new LookupUpdateState(oldState.lookupMap, ImmutableList.of(), oldState.pendingNotices) ); @@ -345,7 +346,7 @@ interface Notice void handle(Map lookupMap); } - private class LoadNotice implements Notice + private static class LoadNotice implements Notice { private final String lookupName; private final LookupExtractorFactoryContainer lookupExtractorFactoryContainer; @@ -398,7 +399,7 @@ public String toString() } } - private class DropNotice implements Notice + private static class DropNotice implements Notice { private final String lookupName; @@ -434,7 +435,7 @@ public String toString() } } - private class LookupUpdateState + private static class LookupUpdateState { private final ImmutableMap lookupMap; private final ImmutableList pendingNotices; diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryConfig.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryConfig.java index 8b4942c23dec..fd4add3da80b 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryConfig.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryConfig.java @@ -20,19 +20,29 @@ package io.druid.query.metadata; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; import org.joda.time.Period; import org.joda.time.format.ISOPeriodFormat; import org.joda.time.format.PeriodFormatter; +import java.util.EnumSet; public class SegmentMetadataQueryConfig { private static final String DEFAULT_PERIOD_STRING = "P1W"; private static final PeriodFormatter ISO_FORMATTER = ISOPeriodFormat.standard(); + private static final EnumSet DEFAULT_ANALYSIS_TYPES = EnumSet.of( + SegmentMetadataQuery.AnalysisType.CARDINALITY, + SegmentMetadataQuery.AnalysisType.INTERVAL, + SegmentMetadataQuery.AnalysisType.MINMAX + ); @JsonProperty private Period defaultHistory = ISO_FORMATTER.parsePeriod(DEFAULT_PERIOD_STRING); + @JsonProperty + private EnumSet defaultAnalysisTypes = DEFAULT_ANALYSIS_TYPES; + public SegmentMetadataQueryConfig(String period) { defaultHistory = ISO_FORMATTER.parsePeriod(period); @@ -46,4 +56,16 @@ public Period getDefaultHistory() { return defaultHistory; } + + public void setDefaultHistory(String period) + { + this.defaultHistory = ISO_FORMATTER.parsePeriod(period); + } + + public EnumSet getDefaultAnalysisTypes() { return defaultAnalysisTypes; } + + public void setDefaultAnalysisTypes(EnumSet defaultAnalysisTypes) + { + this.defaultAnalysisTypes = defaultAnalysisTypes; + } } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index a13f87ccd390..071c6e84d3af 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -38,14 +38,15 @@ import io.druid.java.util.common.guava.MappedSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; +import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; -import io.druid.query.ResultMergeQueryRunner; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.MetricManipulationFn; @@ -88,7 +89,10 @@ public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config) } @Inject - public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config, GenericQueryMetricsFactory queryMetricsFactory) + public SegmentMetadataQueryQueryToolChest( + SegmentMetadataQueryConfig config, + GenericQueryMetricsFactory queryMetricsFactory + ) { this.config = config; this.queryMetricsFactory = queryMetricsFactory; @@ -97,29 +101,30 @@ public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config, Gen @Override public QueryRunner mergeResults(final QueryRunner runner) { - return new ResultMergeQueryRunner(runner) + return new BySegmentSkippingQueryRunner(runner) { @Override public Sequence doRun( QueryRunner baseRunner, - Query query, + QueryPlus queryPlus, Map context ) { + SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(config); + QueryPlus updatedQueryPlus = queryPlus.withQuery(updatedQuery); return new MappedSequence<>( CombiningSequence.create( - baseRunner.run(query, context), - makeOrdering(query), - createMergeFn(query) + baseRunner.run(updatedQueryPlus, context), + makeOrdering(updatedQuery), + createMergeFn(updatedQuery) ), MERGE_TRANSFORM_FN ); } - @Override - protected Ordering makeOrdering(Query query) + private Ordering makeOrdering(SegmentMetadataQuery query) { - if (((SegmentMetadataQuery) query).isMerge()) { + if (query.isMerge()) { // Merge everything always return new Ordering() { @@ -136,17 +141,14 @@ public int compare( return query.getResultOrdering(); // No two elements should be equal, so it should never merge } - @Override - protected BinaryFn createMergeFn(final Query inQ) + private BinaryFn createMergeFn(final SegmentMetadataQuery inQ) { return new BinaryFn() { - private final SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; - @Override public SegmentAnalysis apply(SegmentAnalysis arg1, SegmentAnalysis arg2) { - return mergeAnalyses(arg1, arg2, query.isLenientAggregatorMerge()); + return mergeAnalyses(arg1, arg2, inQ.isLenientAggregatorMerge()); } }; } @@ -187,8 +189,9 @@ public boolean isCacheable(SegmentMetadataQuery query, boolean willMergeRunners) @Override public byte[] computeCacheKey(SegmentMetadataQuery query) { - byte[] includerBytes = query.getToInclude().getCacheKey(); - byte[] analysisTypesBytes = query.getAnalysisTypesCacheKey(); + SegmentMetadataQuery updatedQuery = query.withFinalizedAnalysisTypes(config); + byte[] includerBytes = updatedQuery.getToInclude().getCacheKey(); + byte[] analysisTypesBytes = updatedQuery.getAnalysisTypesCacheKey(); return ByteBuffer.allocate(1 + includerBytes.length + analysisTypesBytes.length) .put(SEGMENT_METADATA_CACHE_PREFIX) .put(includerBytes) @@ -236,7 +239,6 @@ public List filterSegments(SegmentMetadataQuery qu if (!query.isUsingDefaultInterval()) { return segments; } - if (segments.size() <= 1) { return segments; } @@ -404,4 +406,9 @@ public static SegmentAnalysis finalizeAnalysis(SegmentAnalysis analysis) analysis.isRollup() ); } + + public SegmentMetadataQueryConfig getConfig() + { + return this.config; + } } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 475d98453df9..ded0107d0ea5 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -36,6 +36,7 @@ import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -50,7 +51,7 @@ import org.joda.time.Interval; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.CancellationException; @@ -83,10 +84,11 @@ public QueryRunner createRunner(final Segment segment) return new QueryRunner() { @Override - public Sequence run(Query inQ, Map responseContext) + public Sequence run(QueryPlus inQ, Map responseContext) { - SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; - final SegmentAnalyzer analyzer = new SegmentAnalyzer(query.getAnalysisTypes()); + SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) inQ.getQuery()) + .withFinalizedAnalysisTypes(toolChest.getConfig()); + final SegmentAnalyzer analyzer = new SegmentAnalyzer(updatedQuery.getAnalysisTypes()); final Map analyzedColumns = analyzer.analyze(segment); final long numRows = analyzer.numRows(segment); long totalSize = 0; @@ -97,7 +99,7 @@ public Sequence run(Query inQ, Map columns = Maps.newTreeMap(); - ColumnIncluderator includerator = query.getToInclude(); + ColumnIncluderator includerator = updatedQuery.getToInclude(); for (Map.Entry entry : analyzedColumns.entrySet()) { final String columnName = entry.getKey(); final ColumnAnalysis column = entry.getValue(); @@ -109,11 +111,12 @@ public Sequence run(Query inQ, Map retIntervals = query.analyzingInterval() ? Arrays.asList(segment.getDataInterval()) : null; + List retIntervals = updatedQuery.analyzingInterval() ? + Collections.singletonList(segment.getDataInterval()) : null; final Map aggregators; Metadata metadata = null; - if (query.hasAggregators()) { + if (updatedQuery.hasAggregators()) { metadata = segment.asStorageAdapter().getMetadata(); if (metadata != null && metadata.getAggregators() != null) { aggregators = Maps.newHashMap(); @@ -128,7 +131,7 @@ public Sequence run(Query inQ, Map run(Query inQ, Map run(Query inQ, Map run(Query inQ, Map apply(final QueryRunner inp { @Override public Sequence run( - final Query query, + final QueryPlus queryPlus, final Map responseContext ) { + final Query query = queryPlus.getQuery(); final int priority = QueryContexts.getPriority(query); - ListenableFuture> future = queryExecutor.submit( + final QueryPlus threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState(); + final ListenableFuture> future = queryExecutor.submit( new AbstractPrioritizedCallable>(priority) { @Override public Sequence call() throws Exception { return Sequences.simple( - Sequences.toList(input.run(query, responseContext), new ArrayList<>()) + Sequences.toList( + input.run(threadSafeQueryPlus, responseContext), + new ArrayList<>() + ) ); } } @@ -227,7 +235,7 @@ public Sequence call() throws Exception future.cancel(true); throw new QueryInterruptedException(e); } - catch(CancellationException e) { + catch (CancellationException e) { throw new QueryInterruptedException(e); } catch (TimeoutException e) { diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 130025c60770..d185e58a290a 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -32,12 +32,13 @@ import io.druid.query.TableDataSource; import io.druid.query.UnionDataSource; import io.druid.query.filter.DimFilter; +import io.druid.query.metadata.SegmentMetadataQueryConfig; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import java.nio.ByteBuffer; -import java.util.Arrays; +import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -45,11 +46,12 @@ public class SegmentMetadataQuery extends BaseQuery { - /* The SegmentMetadataQuery cache key may contain UTF-8 column name strings. + /** + * The SegmentMetadataQuery cache key may contain UTF-8 column name strings. * Prepend 0xFF before the analysisTypes as a separator to avoid * any potential confusion with string values. */ - public static final byte[] ANALYSIS_TYPES_CACHE_PREFIX = new byte[]{(byte) 0xFF}; + public static final byte[] ANALYSIS_TYPES_CACHE_PREFIX = new byte[] { (byte) 0xFF }; public enum AnalysisType { @@ -77,7 +79,7 @@ public static AnalysisType fromString(String name) public byte[] getCacheKey() { - return new byte[]{(byte) this.ordinal()}; + return new byte[] { (byte) this.ordinal() }; } } @@ -85,12 +87,6 @@ public byte[] getCacheKey() JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT ); - public static final EnumSet DEFAULT_ANALYSIS_TYPES = EnumSet.of( - AnalysisType.CARDINALITY, - AnalysisType.INTERVAL, - AnalysisType.MINMAX - ); - private final ColumnIncluderator toInclude; private final boolean merge; private final boolean usingDefaultInterval; @@ -111,8 +107,8 @@ public SegmentMetadataQuery( { super( dataSource, - (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Arrays.asList(DEFAULT_INTERVAL)) - : querySegmentSpec, + (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(DEFAULT_INTERVAL)) + : querySegmentSpec, false, context ); @@ -124,7 +120,7 @@ public SegmentMetadataQuery( } this.toInclude = toInclude == null ? new AllColumnIncluderator() : toInclude; this.merge = merge == null ? false : merge; - this.analysisTypes = (analysisTypes == null) ? DEFAULT_ANALYSIS_TYPES : analysisTypes; + this.analysisTypes = analysisTypes; Preconditions.checkArgument( dataSource instanceof TableDataSource || dataSource instanceof UnionDataSource, "SegmentMetadataQuery only supports table or union datasource" @@ -229,7 +225,6 @@ public byte[] getAnalysisTypesCacheKey() return bytes.array(); } - @Override public Query withOverriddenContext(Map contextOverride) { @@ -254,18 +249,35 @@ public Query withColumns(ColumnIncluderator includerator) return Druids.SegmentMetadataQueryBuilder.copy(this).toInclude(includerator).build(); } + public SegmentMetadataQuery withFinalizedAnalysisTypes(SegmentMetadataQueryConfig config) + { + if (analysisTypes != null) { + return this; + } + return Druids.SegmentMetadataQueryBuilder + .copy(this) + .analysisTypes(config.getDefaultAnalysisTypes()) + .build(); + } + + @Override + public List getIntervals() + { + return this.getQuerySegmentSpec().getIntervals(); + } + @Override public String toString() { return "SegmentMetadataQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", toInclude=" + toInclude + - ", merge=" + merge + - ", usingDefaultInterval=" + usingDefaultInterval + - ", analysisTypes=" + analysisTypes + - ", lenientAggregatorMerge=" + lenientAggregatorMerge + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", toInclude=" + toInclude + + ", merge=" + merge + + ", usingDefaultInterval=" + usingDefaultInterval + + ", analysisTypes=" + analysisTypes + + ", lenientAggregatorMerge=" + lenientAggregatorMerge + + '}'; } @Override @@ -282,10 +294,10 @@ public boolean equals(Object o) } SegmentMetadataQuery that = (SegmentMetadataQuery) o; return merge == that.merge && - usingDefaultInterval == that.usingDefaultInterval && - lenientAggregatorMerge == that.lenientAggregatorMerge && - Objects.equals(toInclude, that.toInclude) && - Objects.equals(analysisTypes, that.analysisTypes); + usingDefaultInterval == that.usingDefaultInterval && + lenientAggregatorMerge == that.lenientAggregatorMerge && + Objects.equals(toInclude, that.toInclude) && + Objects.equals(analysisTypes, that.analysisTypes); } @Override diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java index af82900fc803..19aa3f4219e6 100644 --- a/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/HotLoopCallee.java @@ -37,7 +37,7 @@ public interface HotLoopCallee * this instance (the instance on which inspectRuntimeShape() is called) is configured. * d. ByteBuffer or similar objects, where byte order matters * e. boolean flags, affecting branch taking - * f. Arrays of objects, meeting conditions any of conditions a-e. + * f. Arrays of objects, meeting any of conditions a-e. */ void inspectRuntimeShape(RuntimeShapeInspector inspector); } diff --git a/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java b/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java index e2d00b05031f..5b590b472315 100644 --- a/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java +++ b/processing/src/main/java/io/druid/query/monomorphicprocessing/SpecializationService.java @@ -100,6 +100,7 @@ public final class SpecializationService * specialization takes some JVM memory (machine code cache, byte code, etc.) */ private static final int maxSpecializations = Integer.getInteger("maxSpecializations", 1000); + private static final AtomicBoolean maxSpecializationsWarningEmitted = new AtomicBoolean(false); private static final ExecutorService classSpecializationExecutor = Execs.singleThreaded("class-specialization-%d"); @@ -124,7 +125,7 @@ public static SpecializationState getSpecializationState( String runtimeShape ) { - return getSpecializationState(prototypeClass, runtimeShape, ImmutableMap., Class>of()); + return getSpecializationState(prototypeClass, runtimeShape, ImmutableMap.of()); } /** @@ -292,6 +293,8 @@ static class WindowedLoopIterationCounter extends SpecializationState impl @Override public T getSpecialized() { + // Returns null because the class is not yet specialized. The purpose of WindowedLoopIterationCounter is to decide + // whether specialization should be done, or not. return null; } @@ -329,7 +332,7 @@ private long addAndGetTotalIterationsOverTheLastHour(long newIterations) } } if (!currentMinutePresent) { - perMinuteIterations.computeIfAbsent(currentMinute, AtomicLong::new).addAndGet(newIterations); + perMinuteIterations.computeIfAbsent(currentMinute, m -> new AtomicLong()).addAndGet(newIterations); totalIterations += newIterations; } return totalIterations; @@ -346,18 +349,33 @@ public void run() // PerPrototypeClassState.specializationStates. But it might be that nobody ever hits even the current // maxSpecializations limit, so implementing cache eviction is an unnecessary complexity. specialized = perPrototypeClassState.prototypeClass.newInstance(); - LOG.warn( - "SpecializationService couldn't make more than [%d] specializations. " - + "Not doing specialization for runtime shape[%s] and class remapping[%s], using the prototype class[%s]", - maxSpecializations, - specializationId.runtimeShape, - specializationId.classRemapping, - perPrototypeClassState.prototypeClass - ); + if (!maxSpecializationsWarningEmitted.get() && maxSpecializationsWarningEmitted.compareAndSet(false, true)) { + LOG.warn( + "SpecializationService couldn't make more than [%d] specializations. " + + "Not doing specialization for runtime shape[%s] and class remapping[%s], using the prototype class[%s]", + maxSpecializations, + specializationId.runtimeShape, + specializationId.classRemapping, + perPrototypeClassState.prototypeClass + ); + } } else if (fakeSpecialize) { specialized = perPrototypeClassState.prototypeClass.newInstance(); + LOG.info( + "Not specializing prototype class[%s] for runtime shape[%s] and class remapping[%s] because " + + "fakeSpecialize=true, using the prototype class instead", + perPrototypeClassState.prototypeClass, + specializationId.runtimeShape, + specializationId.classRemapping + ); } else { specialized = perPrototypeClassState.specialize(specializationId.classRemapping); + LOG.info( + "Specializing prototype class[%s] for runtime shape[%s] and class remapping[%s]", + perPrototypeClassState.prototypeClass, + specializationId.runtimeShape, + specializationId.classRemapping + ); } perPrototypeClassState.specializationStates.put(specializationId, new Specialized<>(specialized)); } diff --git a/processing/src/main/java/io/druid/query/ordering/StringComparators.java b/processing/src/main/java/io/druid/query/ordering/StringComparators.java index 03f15b111d23..5c36917bd93d 100644 --- a/processing/src/main/java/io/druid/query/ordering/StringComparators.java +++ b/processing/src/main/java/io/druid/query/ordering/StringComparators.java @@ -57,11 +57,13 @@ public int compare(String s, String s2) } }).nullsFirst(); + @SuppressWarnings("StringEquality") @Override public int compare(String s, String s2) { // Avoid conversion to bytes for equal references - if(s == s2){ + // Assuming we mostly compare different strings, checking s.equals(s2) will only make the comparison slower. + if (s == s2) { return 0; } @@ -98,6 +100,7 @@ public static class AlphanumericComparator extends StringComparator { // This code is based on https://github.com/amjjd/java-alphanum, see // NOTICE file for more information + @Override public int compare(String str1, String str2) { int[] pos = @@ -369,10 +372,12 @@ private static BigDecimal convertStringToBigDecimal(String input) { public static class NumericComparator extends StringComparator { + @SuppressWarnings("StringEquality") @Override public int compare(String o1, String o2) { // return if o1 and o2 are the same object + // Assuming we mostly compare different strings, checking o1.equals(o2) will only make the comparison slower. if (o1 == o2) { return 0; } diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index d84a7396e3e6..592418e0a8c8 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -42,6 +42,7 @@ import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -348,14 +349,15 @@ public QueryRunner> preMergeQueryDecoration(final Quer { @Override public Sequence> run( - Query> query, Map responseContext + QueryPlus> queryPlus, Map responseContext ) { - SearchQuery searchQuery = (SearchQuery) query; + SearchQuery searchQuery = (SearchQuery) queryPlus.getQuery(); if (searchQuery.getDimensionsFilter() != null) { searchQuery = searchQuery.withDimFilter(searchQuery.getDimensionsFilter().optimize()); + queryPlus = queryPlus.withQuery(searchQuery); } - return runner.run(searchQuery, responseContext); + return runner.run(queryPlus, responseContext); } } , this), config @@ -378,23 +380,24 @@ public SearchThresholdAdjustingQueryRunner( @Override public Sequence> run( - Query> input, + QueryPlus> queryPlus, Map responseContext ) { + Query> input = queryPlus.getQuery(); if (!(input instanceof SearchQuery)) { throw new ISE("Can only handle [%s], got [%s]", SearchQuery.class, input.getClass()); } final SearchQuery query = (SearchQuery) input; if (query.getLimit() < config.getMaxSearchLimit()) { - return runner.run(query, responseContext); + return runner.run(queryPlus, responseContext); } final boolean isBySegment = QueryContexts.isBySegment(query); return Sequences.map( - runner.run(query.withLimit(config.getMaxSearchLimit()), responseContext), + runner.run(queryPlus.withQuery(query.withLimit(config.getMaxSearchLimit())), responseContext), new Function, Result>() { @Override diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java index cde2e4567dbc..9367efb5f08d 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -30,6 +30,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.Result; import io.druid.query.dimension.ColumnSelectorStrategy; @@ -186,10 +187,11 @@ public void updateSearchResultSet( @Override public Sequence> run( - final Query> input, + final QueryPlus> queryPlus, Map responseContext ) { + Query> input = queryPlus.getQuery(); if (!(input instanceof SearchQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class); } diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java index a9971907dd5d..e351d64b029a 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java @@ -162,14 +162,14 @@ public SearchQuery withLimit(int newLimit) public String toString() { return "SearchQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", dimFilter=" + dimFilter + - ", granularity='" + granularity + '\'' + - ", dimensions=" + dimensions + - ", querySpec=" + querySpec + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", limit=" + limit + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", dimFilter=" + dimFilter + + ", granularity='" + granularity + '\'' + + ", dimensions=" + dimensions + + ", querySpec=" + querySpec + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", limit=" + limit + + '}'; } @Override diff --git a/processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java b/processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java index 986e020cdc15..a725bf10fc3d 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java @@ -70,6 +70,7 @@ public byte[] getCacheKey() return ordering.getCacheKey(); } + @Override public String toString() { return String.format("%sSort", ordering.toString()); diff --git a/processing/src/main/java/io/druid/query/select/PagingOffset.java b/processing/src/main/java/io/druid/query/select/PagingOffset.java index c1d98f2bf5c8..57826816a3b5 100644 --- a/processing/src/main/java/io/druid/query/select/PagingOffset.java +++ b/processing/src/main/java/io/druid/query/select/PagingOffset.java @@ -70,16 +70,19 @@ public Ascending(int offset, int threshold) super(offset, threshold); } + @Override public final boolean isDescending() { return false; } + @Override public final int startDelta() { return startOffset; } + @Override public final int current() { return startOffset + counter; @@ -93,16 +96,19 @@ public Descending(int offset, int threshold) super(offset, threshold); } + @Override public final boolean isDescending() { return true; } + @Override public final int startDelta() { return -startOffset - 1; } + @Override public final int current() { return startOffset - counter; diff --git a/processing/src/main/java/io/druid/query/select/SelectQuery.java b/processing/src/main/java/io/druid/query/select/SelectQuery.java index 72059580c4f2..62cfebe6d084 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -145,6 +145,7 @@ public PagingOffset getPagingOffset(String identifier) return pagingSpec.getOffset(identifier, isDescending()); } + @Override public SelectQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { return Druids.SelectQueryBuilder.copy(this).intervals(querySegmentSpec).build(); @@ -156,6 +157,7 @@ public Query> withDataSource(DataSource dataSource) return Druids.SelectQueryBuilder.copy(this).dataSource(dataSource).build(); } + @Override public SelectQuery withOverriddenContext(Map contextOverrides) { Map newContext = computeOverriddenContext(getContext(), contextOverrides); @@ -176,16 +178,16 @@ public SelectQuery withDimFilter(DimFilter dimFilter) public String toString() { return "SelectQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", descending=" + isDescending() + - ", dimFilter=" + dimFilter + - ", granularity=" + granularity + - ", dimensions=" + dimensions + - ", metrics=" + metrics + - ", virtualColumns=" + virtualColumns + - ", pagingSpec=" + pagingSpec + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", descending=" + isDescending() + + ", dimFilter=" + dimFilter + + ", granularity=" + granularity + + ", dimensions=" + dimensions + + ", metrics=" + metrics + + ", virtualColumns=" + virtualColumns + + ", pagingSpec=" + pagingSpec + + '}'; } @Override diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 45948efbe71e..7dabd959ec18 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -43,6 +43,7 @@ import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -348,14 +349,15 @@ public QueryRunner> preMergeQueryDecoration(final Quer { @Override public Sequence> run( - Query> query, Map responseContext + QueryPlus> queryPlus, Map responseContext ) { - SelectQuery selectQuery = (SelectQuery) query; + SelectQuery selectQuery = (SelectQuery) queryPlus.getQuery(); if (selectQuery.getDimensionsFilter() != null) { selectQuery = selectQuery.withDimFilter(selectQuery.getDimensionsFilter().optimize()); + queryPlus = queryPlus.withQuery(selectQuery); } - return runner.run(selectQuery, responseContext); + return runner.run(queryPlus, responseContext); } }, this); } diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java index 19e4bbd1d2ce..2b6d2359d1a2 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java @@ -24,6 +24,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -90,10 +91,11 @@ private SelectQueryRunner(SelectQueryEngine engine, Segment segment) @Override public Sequence> run( - Query> input, + QueryPlus> queryPlus, Map responseContext ) { + Query> input = queryPlus.getQuery(); if (!(input instanceof SelectQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SelectQuery.class); } diff --git a/processing/src/main/java/io/druid/query/spec/LegacySegmentSpec.java b/processing/src/main/java/io/druid/query/spec/LegacySegmentSpec.java index 35af0f9caf51..46a0034e781f 100644 --- a/processing/src/main/java/io/druid/query/spec/LegacySegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/LegacySegmentSpec.java @@ -26,6 +26,7 @@ import org.joda.time.Interval; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -39,7 +40,7 @@ private static List convertValue(Object intervals) if (intervals instanceof String) { intervalStringList = Arrays.asList((((String) intervals).split(","))); } else if (intervals instanceof Interval) { - intervalStringList = Arrays.asList(intervals.toString()); + intervalStringList = Collections.singletonList(intervals.toString()); } else if (intervals instanceof Map) { intervalStringList = (List) ((Map) intervals).get("intervals"); } else if (intervals instanceof List) { diff --git a/processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java index ad61e82b3317..4e78fd9e8765 100644 --- a/processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java +++ b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java @@ -21,7 +21,7 @@ import org.joda.time.Interval; -import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -35,7 +35,7 @@ public static QuerySegmentSpec create(String isoInterval) public static QuerySegmentSpec create(Interval interval) { - return create(Arrays.asList(interval)); + return create(Collections.singletonList(interval)); } public static QuerySegmentSpec create(List intervals) diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index c331ef7556d2..e60c545c5cf3 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.guava.Yielders; import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.Result; import io.druid.query.SegmentDescriptor; @@ -55,9 +56,10 @@ public SpecificSegmentQueryRunner( } @Override - public Sequence run(final Query input, final Map responseContext) + public Sequence run(final QueryPlus input, final Map responseContext) { - final Query query = input.withQuerySegmentSpec(specificSpec); + final QueryPlus queryPlus = input.withQuerySegmentSpec(specificSpec); + final Query query = queryPlus.getQuery(); final Thread currThread = Thread.currentThread(); final String currThreadName = currThread.getName(); @@ -69,7 +71,7 @@ public Sequence run(final Query input, final Map responseC @Override public Sequence get() { - return base.run(query, responseContext); + return base.run(queryPlus, responseContext); } } ); diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java index fe69a0485180..cc2d66139c99 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java @@ -25,7 +25,7 @@ import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; -import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -43,13 +43,13 @@ public SpecificSegmentSpec( @Override public List getIntervals() { - return Arrays.asList(descriptor.getInterval()); + return Collections.singletonList(descriptor.getInterval()); } @Override public QueryRunner lookup(Query query, QuerySegmentWalker walker) { - return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor)); + return walker.getQueryRunnerForSegments(query, Collections.singletonList(descriptor)); } public SegmentDescriptor getDescriptor() { return descriptor; } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index d466cd211406..e97f7cdc11b9 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -37,7 +37,7 @@ import org.joda.time.Interval; import java.nio.ByteBuffer; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -68,8 +68,8 @@ public TimeBoundaryQuery( { super( dataSource, - (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Arrays.asList(MY_Y2K_INTERVAL)) - : querySegmentSpec, + (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(MY_Y2K_INTERVAL)) + : querySegmentSpec, false, context ); @@ -79,7 +79,8 @@ public TimeBoundaryQuery( } @Override - public boolean hasFilters() { + public boolean hasFilters() + { return dimFilter != null; } @@ -123,15 +124,15 @@ public Query> withDataSource(DataSource dataSour public byte[] getCacheKey() { - final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); + final byte[] filterBytes = dimFilter == null ? new byte[] {} : dimFilter.getCacheKey(); final byte[] boundBytes = StringUtils.toUtf8(bound); final byte delimiter = (byte) 0xff; return ByteBuffer.allocate(2 + boundBytes.length + filterBytes.length) - .put(CACHE_TYPE_ID) - .put(boundBytes) - .put(delimiter) - .put(filterBytes) - .array(); + .put(CACHE_TYPE_ID) + .put(boundBytes) + .put(delimiter) + .put(filterBytes) + .array(); } public Iterable> buildResult(DateTime timestamp, DateTime min, DateTime max) @@ -208,12 +209,12 @@ boolean isMaxTime() public String toString() { return "TimeBoundaryQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", duration=" + getDuration() + - ", bound=" + bound + - ", dimFilter=" + dimFilter + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", duration=" + getDuration() + + ", bound=" + bound + + ", dimFilter=" + dimFilter + + '}'; } @Override diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a6569baed713..8c7abc576d29 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -32,9 +32,10 @@ import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.GenericQueryMetricsFactory; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -109,13 +110,15 @@ public QueryRunner> mergeResults( { @Override protected Sequence> doRun( - QueryRunner> baseRunner, Query> input, Map context + QueryRunner> baseRunner, + QueryPlus> input, + Map context ) { - TimeBoundaryQuery query = (TimeBoundaryQuery) input; + TimeBoundaryQuery query = (TimeBoundaryQuery) input.getQuery(); return Sequences.simple( query.mergeResults( - Sequences.toList(baseRunner.run(query, context), Lists.>newArrayList()) + Sequences.toList(baseRunner.run(input, context), Lists.>newArrayList()) ) ); } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 913933ffd333..1c51392086f2 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerHelper; @@ -94,6 +95,7 @@ public TimeBoundaryQueryRunner(Segment segment) this.adapter = segment.asStorageAdapter(); this.skipToFirstMatching = new Function>() { + @SuppressWarnings("ArgumentParameterSwap") @Override public Result apply(Cursor cursor) { @@ -131,10 +133,11 @@ private DateTime getTimeBoundary(StorageAdapter adapter, TimeBoundaryQuery legac @Override public Sequence> run( - final Query> input, + final QueryPlus> queryPlus, final Map responseContext ) { + Query> input = queryPlus.getQuery(); if (!(input instanceof TimeBoundaryQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeBoundaryQuery.class); } diff --git a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java index addaac5a76f5..d8a015bd2a39 100644 --- a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java @@ -42,13 +42,13 @@ public void query(TimeseriesQuery query) @Override public void numMetrics(TimeseriesQuery query) { - builder.setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); } @Override public void numComplexMetrics(TimeseriesQuery query) { int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); - builder.setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); } } diff --git a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java index f3f99d35645c..23277d65420f 100644 --- a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java @@ -22,8 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; +import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; +@LazySingleton public class DefaultTimeseriesQueryMetricsFactory implements TimeseriesQueryMetricsFactory { private static final TimeseriesQueryMetricsFactory INSTANCE = @@ -42,7 +45,7 @@ public static TimeseriesQueryMetricsFactory instance() private final ObjectMapper jsonMapper; @Inject - public DefaultTimeseriesQueryMetricsFactory(ObjectMapper jsonMapper) + public DefaultTimeseriesQueryMetricsFactory(@Json ObjectMapper jsonMapper) { this.jsonMapper = jsonMapper; } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java index 328400291348..82e6c74e52b5 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -129,6 +129,7 @@ public boolean isSkipEmptyBuckets() return getContextBoolean("skipEmptyBuckets", false); } + @Override public TimeseriesQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { return Druids.TimeseriesQueryBuilder.copy(this).intervals(querySegmentSpec).build(); @@ -140,6 +141,7 @@ public Query> withDataSource(DataSource dataSource return Druids.TimeseriesQueryBuilder.copy(this).dataSource(dataSource).build(); } + @Override public TimeseriesQuery withOverriddenContext(Map contextOverrides) { Map newContext = computeOverriddenContext(getContext(), contextOverrides); @@ -160,16 +162,16 @@ public TimeseriesQuery withPostAggregatorSpecs(final List postAg public String toString() { return "TimeseriesQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", descending=" + isDescending() + - ", virtualColumns=" + virtualColumns + - ", dimFilter=" + dimFilter + - ", granularity='" + granularity + '\'' + - ", aggregatorSpecs=" + aggregatorSpecs + - ", postAggregatorSpecs=" + postAggregatorSpecs + - ", context=" + getContext() + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", descending=" + isDescending() + + ", virtualColumns=" + virtualColumns + + ", dimFilter=" + dimFilter + + ", granularity='" + granularity + '\'' + + ", aggregatorSpecs=" + aggregatorSpecs + + ", postAggregatorSpecs=" + postAggregatorSpecs + + ", context=" + getContext() + + '}'; } @Override @@ -186,10 +188,10 @@ public boolean equals(final Object o) } final TimeseriesQuery that = (TimeseriesQuery) o; return Objects.equals(virtualColumns, that.virtualColumns) && - Objects.equals(dimFilter, that.dimFilter) && - Objects.equals(granularity, that.granularity) && - Objects.equals(aggregatorSpecs, that.aggregatorSpecs) && - Objects.equals(postAggregatorSpecs, that.postAggregatorSpecs); + Objects.equals(dimFilter, that.dimFilter) && + Objects.equals(granularity, that.granularity) && + Objects.equals(aggregatorSpecs, that.aggregatorSpecs) && + Objects.equals(postAggregatorSpecs, that.postAggregatorSpecs); } @Override diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java index d84f1542994b..d35cf18dbad2 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java @@ -19,6 +19,17 @@ package io.druid.query.timeseries; +/** + * Implementations could be injected using + * + * PolyBind + * .optionBinder(binder, Key.get(TimeseriesQueryMetricsFactory.class)) + * .addBinding("myCustomTimeseriesQueryMetricsFactory") + * .to(MyCustomTimeseriesQueryMetricsFactory.class); + * + * And then setting property: + * druid.query.timeseries.queryMetricsFactory=myCustomTimeseriesQueryMetricsFactory + */ public interface TimeseriesQueryMetricsFactory { diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 11f9209bedde..473775b71bc0 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -33,6 +33,7 @@ import io.druid.query.CacheStrategy; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -92,14 +93,14 @@ public QueryRunner> mergeResults( @Override public Sequence> doRun( QueryRunner> baseRunner, - Query> query, + QueryPlus> queryPlus, Map context ) { return super.doRun( baseRunner, // Don't do post aggs until makePostComputeManipulatorFn() is called - ((TimeseriesQuery) query).withPostAggregatorSpecs(ImmutableList.of()), + queryPlus.withQuery(((TimeseriesQuery) queryPlus.getQuery()).withPostAggregatorSpecs(ImmutableList.of())), context ); } @@ -234,14 +235,15 @@ public QueryRunner> preMergeQueryDecoration(final { @Override public Sequence> run( - Query> query, Map responseContext + QueryPlus> queryPlus, Map responseContext ) { - TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; + TimeseriesQuery timeseriesQuery = (TimeseriesQuery) queryPlus.getQuery(); if (timeseriesQuery.getDimensionsFilter() != null) { timeseriesQuery = timeseriesQuery.withDimFilter(timeseriesQuery.getDimensionsFilter().optimize()); + queryPlus = queryPlus.withQuery(timeseriesQuery); } - return runner.run(timeseriesQuery, responseContext); + return runner.run(queryPlus, responseContext); } }, this); } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index e6ff92b16e21..79922df83e91 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -24,6 +24,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -91,10 +92,11 @@ private TimeseriesQueryRunner(TimeseriesQueryEngine engine, StorageAdapter adapt @Override public Sequence> run( - Query> input, + QueryPlus> queryPlus, Map responseContext ) { + Query> input = queryPlus.getQuery(); if (!(input instanceof TimeseriesQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeseriesQuery.class); } diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java index 2be769b6c1fd..9dadbbcfd480 100644 --- a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java @@ -44,25 +44,25 @@ public void query(TopNQuery query) @Override public void threshold(TopNQuery query) { - builder.setDimension("threshold", String.valueOf(query.getThreshold())); + setDimension("threshold", String.valueOf(query.getThreshold())); } @Override public void dimension(TopNQuery query) { - builder.setDimension("dimension", query.getDimensionSpec().getDimension()); + setDimension("dimension", query.getDimensionSpec().getDimension()); } @Override public void numMetrics(TopNQuery query) { - builder.setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); } @Override public void numComplexMetrics(TopNQuery query) { int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); - builder.setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); } } diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java index c56e2321bf38..926b83c1248f 100644 --- a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java @@ -22,8 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; +import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.Json; import io.druid.jackson.DefaultObjectMapper; +@LazySingleton public class DefaultTopNQueryMetricsFactory implements TopNQueryMetricsFactory { private static final TopNQueryMetricsFactory INSTANCE = new DefaultTopNQueryMetricsFactory(new DefaultObjectMapper()); @@ -41,7 +44,7 @@ public static TopNQueryMetricsFactory instance() private final ObjectMapper jsonMapper; @Inject - public DefaultTopNQueryMetricsFactory(ObjectMapper jsonMapper) + public DefaultTopNQueryMetricsFactory(@Json ObjectMapper jsonMapper) { this.jsonMapper = jsonMapper; } diff --git a/processing/src/main/java/io/druid/query/topn/Historical1AggPooledTopNScanner.java b/processing/src/main/java/io/druid/query/topn/Historical1AggPooledTopNScanner.java new file mode 100644 index 000000000000..fc1691a10786 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/Historical1AggPooledTopNScanner.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.topn; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.historical.HistoricalCursor; +import io.druid.segment.historical.HistoricalDimensionSelector; + +import java.nio.ByteBuffer; + +public interface Historical1AggPooledTopNScanner< + DimensionSelectorType extends HistoricalDimensionSelector, + MetricSelectorType extends ColumnValueSelector, + BufferAggregatorType extends BufferAggregator> +{ + /** + * @param aggregatorSize number of bytes required by aggregator for a single aggregation + * @param positions a cache for positions in resultsBuffer, where specific (indexed) dimension values are aggregated + * @return number of scanned rows, i. e. number of steps made with the given cursor + */ + long scanAndAggregate( + DimensionSelectorType dimensionSelector, + MetricSelectorType metricSelector, + BufferAggregatorType aggregator, + int aggregatorSize, + HistoricalCursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ); +} diff --git a/processing/src/main/java/io/druid/query/topn/Historical1SimpleDoubleAggPooledTopNScannerPrototype.java b/processing/src/main/java/io/druid/query/topn/Historical1SimpleDoubleAggPooledTopNScannerPrototype.java new file mode 100644 index 000000000000..e7d8aaf52a67 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/Historical1SimpleDoubleAggPooledTopNScannerPrototype.java @@ -0,0 +1,74 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.topn; + +import io.druid.query.aggregation.SimpleDoubleBufferAggregator; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.Offset; +import io.druid.segment.historical.HistoricalCursor; +import io.druid.segment.historical.HistoricalDimensionSelector; +import io.druid.segment.historical.HistoricalFloatColumnSelector; + +import java.nio.ByteBuffer; + +public class Historical1SimpleDoubleAggPooledTopNScannerPrototype + implements Historical1AggPooledTopNScanner< + HistoricalDimensionSelector, + HistoricalFloatColumnSelector, + SimpleDoubleBufferAggregator + > +{ + @Override + public long scanAndAggregate( + HistoricalDimensionSelector dimensionSelector, + HistoricalFloatColumnSelector metricSelector, + SimpleDoubleBufferAggregator aggregator, + int aggregatorSize, + HistoricalCursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ) + { + // See TopNUtils.copyOffset() for explanation + Offset offset = (Offset) TopNUtils.copyOffset(cursor); + long scannedRows = 0; + int positionToAllocate = 0; + while (offset.withinBounds() && !Thread.currentThread().isInterrupted()) { + int rowNum = offset.getOffset(); + double metric = metricSelector.get(rowNum); + final IndexedInts dimValues = dimensionSelector.getRow(rowNum); + final int dimSize = dimValues.size(); + for (int i = 0; i < dimSize; i++) { + int dimIndex = dimValues.get(i); + int position = positions[dimIndex]; + if (position >= 0) { + aggregator.aggregate(resultsBuffer, position, metric); + } else if (position == TopNAlgorithm.INIT_POSITION_VALUE) { + positions[dimIndex] = positionToAllocate; + aggregator.putFirst(resultsBuffer, positionToAllocate, metric); + positionToAllocate += aggregatorSize; + } + } + scannedRows++; + offset.increment(); + } + return scannedRows; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype.java b/processing/src/main/java/io/druid/query/topn/HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype.java new file mode 100644 index 000000000000..91f77c4a8091 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype.java @@ -0,0 +1,68 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.topn; + +import io.druid.query.aggregation.SimpleDoubleBufferAggregator; +import io.druid.segment.data.Offset; +import io.druid.segment.historical.HistoricalCursor; +import io.druid.segment.historical.HistoricalFloatColumnSelector; +import io.druid.segment.historical.SingleValueHistoricalDimensionSelector; + +import java.nio.ByteBuffer; + +public class HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype + implements Historical1AggPooledTopNScanner< + SingleValueHistoricalDimensionSelector, + HistoricalFloatColumnSelector, + SimpleDoubleBufferAggregator + > +{ + @Override + public long scanAndAggregate( + SingleValueHistoricalDimensionSelector dimensionSelector, + HistoricalFloatColumnSelector metricSelector, + SimpleDoubleBufferAggregator aggregator, + int aggregatorSize, + HistoricalCursor cursor, + int[] positions, + ByteBuffer resultsBuffer + ) + { + // See TopNUtils.copyOffset() for explanation + Offset offset = (Offset) TopNUtils.copyOffset(cursor); + long scannedRows = 0; + int positionToAllocate = 0; + while (offset.withinBounds() && !Thread.currentThread().isInterrupted()) { + int rowNum = offset.getOffset(); + int dimIndex = dimensionSelector.getRowValue(rowNum); + int position = positions[dimIndex]; + if (position >= 0) { + aggregator.aggregate(resultsBuffer, position, metricSelector.get(rowNum)); + } else if (position == TopNAlgorithm.INIT_POSITION_VALUE) { + positions[dimIndex] = positionToAllocate; + aggregator.putFirst(resultsBuffer, positionToAllocate, metricSelector.get(rowNum)); + positionToAllocate += aggregatorSize; + } + scannedRows++; + offset.increment(); + } + return scannedRows; + } +} diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index b14cfd8fb7f9..4d5974e2e597 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.java.util.common.Pair; @@ -28,6 +29,7 @@ import io.druid.query.BaseQuery; import io.druid.query.ColumnSelectorPlus; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SimpleDoubleBufferAggregator; import io.druid.query.monomorphicprocessing.SpecializationService; import io.druid.query.monomorphicprocessing.SpecializationState; import io.druid.query.monomorphicprocessing.StringRuntimeShape; @@ -36,29 +38,156 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.Offset; +import io.druid.segment.historical.HistoricalCursor; +import io.druid.segment.historical.HistoricalDimensionSelector; +import io.druid.segment.historical.HistoricalFloatColumnSelector; +import io.druid.segment.historical.SingleValueHistoricalDimensionSelector; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; /** */ public class PooledTopNAlgorithm extends BaseTopNAlgorithm { - /** Non-final fields for testing, see TopNQueryRunnerTest */ - @VisibleForTesting - static boolean specializeGeneric1AggPooledTopN = + private static boolean specializeGeneric1AggPooledTopN = !Boolean.getBoolean("dontSpecializeGeneric1AggPooledTopN"); - @VisibleForTesting - static boolean specializeGeneric2AggPooledTopN = + private static boolean specializeGeneric2AggPooledTopN = !Boolean.getBoolean("dontSpecializeGeneric2AggPooledTopN"); + private static boolean specializeHistorical1SimpleDoubleAggPooledTopN = + !Boolean.getBoolean("dontSpecializeHistorical1SimpleDoubleAggPooledTopN"); + private static boolean specializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN = + !Boolean.getBoolean("dontSpecializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN"); + + /** See TopNQueryRunnerTest */ + @VisibleForTesting + static void setSpecializeGeneric1AggPooledTopN(boolean value) + { + PooledTopNAlgorithm.specializeGeneric1AggPooledTopN = value; + computeSpecializedScanAndAggregateImplementations(); + } + + @VisibleForTesting + static void setSpecializeGeneric2AggPooledTopN(boolean value) + { + PooledTopNAlgorithm.specializeGeneric2AggPooledTopN = value; + computeSpecializedScanAndAggregateImplementations(); + } + + @VisibleForTesting + static void setSpecializeHistorical1SimpleDoubleAggPooledTopN(boolean value) + { + PooledTopNAlgorithm.specializeHistorical1SimpleDoubleAggPooledTopN = value; + computeSpecializedScanAndAggregateImplementations(); + } + + @VisibleForTesting + static void setSpecializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN(boolean value) + { + PooledTopNAlgorithm.specializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN = value; + computeSpecializedScanAndAggregateImplementations(); + } private static final Generic1AggPooledTopNScanner defaultGeneric1AggScanner = new Generic1AggPooledTopNScannerPrototype(); private static final Generic2AggPooledTopNScanner defaultGeneric2AggScanner = new Generic2AggPooledTopNScannerPrototype(); + private static final Historical1AggPooledTopNScanner defaultHistorical1SimpleDoubleAggScanner = + new Historical1SimpleDoubleAggPooledTopNScannerPrototype(); + private static final + Historical1AggPooledTopNScanner defaultHistoricalSingleValueDimSelector1SimpleDoubleAggScanner = + new HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype(); + + private interface ScanAndAggregate + { + /** + * If this implementation of ScanAndAggregate is executable with the given parameters, run it and return true. + * Otherwise return false (scanning and aggregation is not done). + */ + boolean scanAndAggregate( + final PooledTopNParams params, + final int[] positions, + final BufferAggregator[] theAggregators + ); + } + + private static final List specializedScanAndAggregateImplementations = new ArrayList<>(); + static { + computeSpecializedScanAndAggregateImplementations(); + } + + private static void computeSpecializedScanAndAggregateImplementations() + { + specializedScanAndAggregateImplementations.clear(); + // The order of the following `if` blocks matters, "more specialized" implementations go first + if (specializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN) { + specializedScanAndAggregateImplementations.add((params, positions, theAggregators) -> { + if (theAggregators.length == 1) { + BufferAggregator aggregator = theAggregators[0]; + final Cursor cursor = params.getCursor(); + if (cursor instanceof HistoricalCursor && aggregator instanceof SimpleDoubleBufferAggregator) { + if (params.getDimSelector() instanceof SingleValueHistoricalDimensionSelector && + ((SimpleDoubleBufferAggregator) aggregator).getSelector() instanceof HistoricalFloatColumnSelector) { + scanAndAggregateHistorical1SimpleDoubleAgg( + params, + positions, + (SimpleDoubleBufferAggregator) aggregator, + (HistoricalCursor) cursor, + defaultHistoricalSingleValueDimSelector1SimpleDoubleAggScanner + ); + return true; + } + } + } + return false; + }); + } + if (specializeHistorical1SimpleDoubleAggPooledTopN) { + specializedScanAndAggregateImplementations.add((params, positions, theAggregators) -> { + if (theAggregators.length == 1) { + BufferAggregator aggregator = theAggregators[0]; + final Cursor cursor = params.getCursor(); + if (cursor instanceof HistoricalCursor && aggregator instanceof SimpleDoubleBufferAggregator) { + if (params.getDimSelector() instanceof HistoricalDimensionSelector && + ((SimpleDoubleBufferAggregator) aggregator).getSelector() instanceof HistoricalFloatColumnSelector) { + scanAndAggregateHistorical1SimpleDoubleAgg( + params, + positions, + (SimpleDoubleBufferAggregator) aggregator, + (HistoricalCursor) cursor, + defaultHistorical1SimpleDoubleAggScanner + ); + return true; + } + } + } + return false; + }); + } + if (specializeGeneric1AggPooledTopN) { + specializedScanAndAggregateImplementations.add((params, positions, theAggregators) -> { + if (theAggregators.length == 1) { + scanAndAggregateGeneric1Agg(params, positions, theAggregators[0], params.getCursor()); + return true; + } + return false; + }); + } + if (specializeGeneric2AggPooledTopN) { + specializedScanAndAggregateImplementations.add((params, positions, theAggregators) -> { + if (theAggregators.length == 2) { + scanAndAggregateGeneric2Agg(params, positions, theAggregators, params.getCursor()); + return true; + } + return false; + }); + } + } - private final Capabilities capabilities; private final TopNQuery query; private final StupidPool bufferPool; private static final int AGG_UNROLL_COUNT = 8; // Must be able to fit loop below @@ -70,8 +199,6 @@ public PooledTopNAlgorithm( ) { super(capabilities); - - this.capabilities = capabilities; this.query = query; this.bufferPool = bufferPool; } @@ -193,17 +320,45 @@ protected void scanAndAggregate( final int numProcessed ) { - final Cursor cursor = params.getCursor(); - if (specializeGeneric1AggPooledTopN && theAggregators.length == 1) { - scanAndAggregateGeneric1Agg(params, positions, theAggregators[0], cursor); - } else if (specializeGeneric2AggPooledTopN && theAggregators.length == 2) { - scanAndAggregateGeneric2Agg(params, positions, theAggregators, cursor); - } else { - scanAndAggregateDefault(params, positions, theAggregators); + for (ScanAndAggregate specializedScanAndAggregate : specializedScanAndAggregateImplementations) { + if (specializedScanAndAggregate.scanAndAggregate(params, positions, theAggregators)) { + BaseQuery.checkInterrupted(); + return; + } } + scanAndAggregateDefault(params, positions, theAggregators); BaseQuery.checkInterrupted(); } + private static void scanAndAggregateHistorical1SimpleDoubleAgg( + PooledTopNParams params, + int[] positions, + SimpleDoubleBufferAggregator aggregator, + HistoricalCursor cursor, + Historical1AggPooledTopNScanner prototypeScanner + ) + { + String runtimeShape = StringRuntimeShape.of(aggregator); + SpecializationState specializationState = + SpecializationService.getSpecializationState( + prototypeScanner.getClass(), + runtimeShape, + ImmutableMap.of(Offset.class, cursor.getOffset().getClass()) + ); + Historical1AggPooledTopNScanner scanner = specializationState.getSpecializedOrDefault(prototypeScanner); + + long scannedRows = scanner.scanAndAggregate( + (HistoricalDimensionSelector) params.getDimSelector(), + aggregator.getSelector(), + aggregator, + params.getAggregatorSizes()[0], + cursor, + positions, + params.getResultsBuf() + ); + specializationState.accountLoopIterations(scannedRows); + } + private static void scanAndAggregateGeneric1Agg( PooledTopNParams params, int[] positions, @@ -314,6 +469,7 @@ private static void scanAndAggregateDefault( dimValues.get(6), currentPosition ); + // fall through case 6: currentPosition = aggregateDimValue( positions, @@ -326,6 +482,7 @@ private static void scanAndAggregateDefault( dimValues.get(5), currentPosition ); + // fall through case 5: currentPosition = aggregateDimValue( positions, @@ -338,6 +495,7 @@ private static void scanAndAggregateDefault( dimValues.get(4), currentPosition ); + // fall through case 4: currentPosition = aggregateDimValue( positions, @@ -350,6 +508,7 @@ private static void scanAndAggregateDefault( dimValues.get(3), currentPosition ); + // fall through case 3: currentPosition = aggregateDimValue( positions, @@ -362,6 +521,7 @@ private static void scanAndAggregateDefault( dimValues.get(2), currentPosition ); + // fall through case 2: currentPosition = aggregateDimValue( positions, @@ -374,6 +534,7 @@ private static void scanAndAggregateDefault( dimValues.get(1), currentPosition ); + // fall through case 1: currentPosition = aggregateDimValue( positions, @@ -513,16 +674,22 @@ private static int aggregateDimValue( switch(aggExtra) { case 7: theAggregators[6].aggregate(resultsBuf, position + aggregatorOffsets[6]); + // fall through case 6: theAggregators[5].aggregate(resultsBuf, position + aggregatorOffsets[5]); + // fall through case 5: theAggregators[4].aggregate(resultsBuf, position + aggregatorOffsets[4]); + // fall through case 4: theAggregators[3].aggregate(resultsBuf, position + aggregatorOffsets[3]); + // fall through case 3: theAggregators[2].aggregate(resultsBuf, position + aggregatorOffsets[2]); + // fall through case 2: theAggregators[1].aggregate(resultsBuf, position + aggregatorOffsets[1]); + // fall through case 1: theAggregators[0].aggregate(resultsBuf, position + aggregatorOffsets[0]); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java index 5a97c486c383..4cfc39ea085c 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNLexicographicResultBuilder.java @@ -96,16 +96,22 @@ public TopNResultBuilder addEntry( switch (extra) { case 7: metricValues.put(aggFactoryNames[6], metricVals[6]); + // fall through case 6: metricValues.put(aggFactoryNames[5], metricVals[5]); + // fall through case 5: metricValues.put(aggFactoryNames[4], metricVals[4]); + // fall through case 4: metricValues.put(aggFactoryNames[3], metricVals[3]); + // fall through case 3: metricValues.put(aggFactoryNames[2], metricVals[2]); + // fall through case 2: metricValues.put(aggFactoryNames[1], metricVals[1]); + // fall through case 1: metricValues.put(aggFactoryNames[0], metricVals[0]); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java index 0fe43daa2577..fd5aa3f3cb44 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNNumericResultBuilder.java @@ -133,16 +133,22 @@ public TopNNumericResultBuilder addEntry( switch (extra) { case 7: metricValues.put(aggFactoryNames[6], metricVals[6]); + // fall through case 6: metricValues.put(aggFactoryNames[5], metricVals[5]); + // fall through case 5: metricValues.put(aggFactoryNames[4], metricVals[4]); + // fall through case 4: metricValues.put(aggFactoryNames[3], metricVals[3]); + // fall through case 3: metricValues.put(aggFactoryNames[2], metricVals[2]); + // fall through case 2: metricValues.put(aggFactoryNames[1], metricVals[1]); + // fall through case 1: metricValues.put(aggFactoryNames[0], metricVals[0]); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQuery.java b/processing/src/main/java/io/druid/query/topn/TopNQuery.java index df7583ebd760..4bdbf70b7a20 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQuery.java @@ -83,8 +83,8 @@ public TopNQuery( this.postAggregatorSpecs = Queries.prepareAggregations( this.aggregatorSpecs, postAggregatorSpecs == null - ? ImmutableList.of() - : postAggregatorSpecs + ? ImmutableList.of() + : postAggregatorSpecs ); Preconditions.checkNotNull(dimensionSpec, "dimensionSpec can't be null"); @@ -168,6 +168,7 @@ public void initTopNAlgorithmSelector(TopNAlgorithmSelector selector) topNMetricSpec.initTopNAlgorithmSelector(selector); } + @Override public TopNQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { return new TopNQueryBuilder(this).intervals(querySegmentSpec).build(); @@ -199,6 +200,7 @@ public TopNQuery withThreshold(int threshold) return new TopNQueryBuilder(this).threshold(threshold).build(); } + @Override public TopNQuery withOverriddenContext(Map contextOverrides) { return new TopNQueryBuilder(this).context(computeOverriddenContext(getContext(), contextOverrides)).build(); @@ -213,17 +215,17 @@ public TopNQuery withDimFilter(DimFilter dimFilter) public String toString() { return "TopNQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", dimensionSpec=" + dimensionSpec + - ", topNMetricSpec=" + topNMetricSpec + - ", threshold=" + threshold + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", virtualColumns=" + virtualColumns + - ", dimFilter=" + dimFilter + - ", granularity='" + granularity + '\'' + - ", aggregatorSpecs=" + aggregatorSpecs + - ", postAggregatorSpecs=" + postAggregatorSpecs + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", dimensionSpec=" + dimensionSpec + + ", topNMetricSpec=" + topNMetricSpec + + ", threshold=" + threshold + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", virtualColumns=" + virtualColumns + + ", dimFilter=" + dimFilter + + ", granularity='" + granularity + '\'' + + ", aggregatorSpecs=" + aggregatorSpecs + + ", postAggregatorSpecs=" + postAggregatorSpecs + + '}'; } @Override @@ -240,13 +242,13 @@ public boolean equals(final Object o) } final TopNQuery topNQuery = (TopNQuery) o; return threshold == topNQuery.threshold && - Objects.equals(virtualColumns, topNQuery.virtualColumns) && - Objects.equals(dimensionSpec, topNQuery.dimensionSpec) && - Objects.equals(topNMetricSpec, topNQuery.topNMetricSpec) && - Objects.equals(dimFilter, topNQuery.dimFilter) && - Objects.equals(granularity, topNQuery.granularity) && - Objects.equals(aggregatorSpecs, topNQuery.aggregatorSpecs) && - Objects.equals(postAggregatorSpecs, topNQuery.postAggregatorSpecs); + Objects.equals(virtualColumns, topNQuery.virtualColumns) && + Objects.equals(dimensionSpec, topNQuery.dimensionSpec) && + Objects.equals(topNMetricSpec, topNQuery.topNMetricSpec) && + Objects.equals(dimFilter, topNQuery.dimFilter) && + Objects.equals(granularity, topNQuery.granularity) && + Objects.equals(aggregatorSpecs, topNQuery.aggregatorSpecs) && + Objects.equals(postAggregatorSpecs, topNQuery.postAggregatorSpecs); } @Override diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java index e0ba452e61ec..8f791d0cb258 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java @@ -19,6 +19,17 @@ package io.druid.query.topn; +/** + * Implementations could be injected using + * + * PolyBind + * .optionBinder(binder, Key.get(TopNQueryMetricsFactory.class)) + * .addBinding("myCustomTopNQueryMetricsFactory") + * .to(MyCustomTopNQueryMetricsFactory.class); + * + * And then setting property: + * druid.query.topN.queryMetricsFactory=myCustomTopNQueryMetricsFactory + */ public interface TopNQueryMetricsFactory { diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 7d9be82c6fb4..12e489a85a65 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -37,6 +37,7 @@ import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; import io.druid.query.QueryContexts; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -416,26 +417,27 @@ public QueryRunner> preMergeQueryDecoration(final QueryR { @Override public Sequence> run( - Query> query, Map responseContext + QueryPlus> queryPlus, Map responseContext ) { - TopNQuery topNQuery = (TopNQuery) query; + TopNQuery topNQuery = (TopNQuery) queryPlus.getQuery(); if (topNQuery.getDimensionsFilter() != null) { topNQuery = topNQuery.withDimFilter(topNQuery.getDimensionsFilter().optimize()); } final TopNQuery delegateTopNQuery = topNQuery; if (TopNQueryEngine.canApplyExtractionInPost(delegateTopNQuery)) { final DimensionSpec dimensionSpec = delegateTopNQuery.getDimensionSpec(); - return runner.run( + QueryPlus> delegateQueryPlus = queryPlus.withQuery( delegateTopNQuery.withDimensionSpec( new DefaultDimensionSpec( dimensionSpec.getDimension(), dimensionSpec.getOutputName() ) - ), responseContext + ) ); + return runner.run(delegateQueryPlus, responseContext); } else { - return runner.run(delegateTopNQuery, responseContext); + return runner.run(queryPlus.withQuery(delegateTopNQuery), responseContext); } } } @@ -455,12 +457,12 @@ public QueryRunner> postMergeQueryDecoration(final Query @Override public Sequence> run( - final Query> query, final Map responseContext + final QueryPlus> queryPlus, final Map responseContext ) { // thresholdRunner.run throws ISE if query is not TopNQuery - final Sequence> resultSequence = thresholdRunner.run(query, responseContext); - final TopNQuery topNQuery = (TopNQuery) query; + final Sequence> resultSequence = thresholdRunner.run(queryPlus, responseContext); + final TopNQuery topNQuery = (TopNQuery) queryPlus.getQuery(); if (!TopNQueryEngine.canApplyExtractionInPost(topNQuery)) { return resultSequence; } else { @@ -521,10 +523,11 @@ public ThresholdAdjustingQueryRunner( @Override public Sequence> run( - Query> input, + QueryPlus> queryPlus, Map responseContext ) { + Query> input = queryPlus.getQuery(); if (!(input instanceof TopNQuery)) { throw new ISE("Can only handle [%s], got [%s]", TopNQuery.class, input.getClass()); } @@ -532,13 +535,13 @@ public Sequence> run( final TopNQuery query = (TopNQuery) input; final int minTopNThreshold = query.getContextValue("minTopNThreshold", config.getMinTopNThreshold()); if (query.getThreshold() > minTopNThreshold) { - return runner.run(query, responseContext); + return runner.run(queryPlus, responseContext); } final boolean isBySegment = QueryContexts.isBySegment(query); return Sequences.map( - runner.run(query.withThreshold(minTopNThreshold), responseContext), + runner.run(queryPlus.withQuery(query.withThreshold(minTopNThreshold)), responseContext), new Function, Result>() { @Override diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java index 35af25e12a60..2e392f0b75e9 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -25,7 +25,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; import io.druid.query.ChainedExecutionQueryRunner; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -65,15 +65,15 @@ public QueryRunner> createRunner(final Segment segment) { @Override public Sequence> run( - Query> input, + QueryPlus> input, Map responseContext ) { - if (!(input instanceof TopNQuery)) { + if (!(input.getQuery() instanceof TopNQuery)) { throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TopNQuery.class); } - return queryEngine.query((TopNQuery) input, segment.asStorageAdapter()); + return queryEngine.query((TopNQuery) input.getQuery(), segment.asStorageAdapter()); } }; diff --git a/processing/src/main/java/io/druid/query/topn/TopNUtils.java b/processing/src/main/java/io/druid/query/topn/TopNUtils.java new file mode 100644 index 000000000000..08afbcb3afdc --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNUtils.java @@ -0,0 +1,45 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.topn; + +import io.druid.segment.historical.HistoricalCursor; + +final class TopNUtils +{ + /** + * Returns Object, so javac couldn't remove cast in methods like + * {@link HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype#scanAndAggregate}. That cast is + * needed, because when TopNScannerPrototype is specialized, occurrences of {@link io.druid.segment.data.Offset} are + * replaced with the specific Offset subtype in the TopNScannerPrototype bytecode, via {@link + * io.druid.query.monomorphicprocessing.SpecializationService#getSpecializationState(Class, String, + * com.google.common.collect.ImmutableMap)}, providing ImmutableMap.of(Offset.class, specificOffsetSubtype) as the + * classRemapping argument. + * + * Casting to the specific Offset subtype helps Hotspot JIT (OpenJDK 8) to generate better assembly. It shouldn't be + * so, because the Offset subtype is still always the same (otherwise cast wouldn't be possible), so JIT should + * generate equivalent code. In OpenJDK 9 Hotspot could be improved and this "casting hack" is not needed anymore. + */ + static Object copyOffset(HistoricalCursor cursor) + { + return cursor.getOffset().clone(); + } + + private TopNUtils() {} +} diff --git a/processing/src/main/java/io/druid/segment/BitmapOffset.java b/processing/src/main/java/io/druid/segment/BitmapOffset.java index 8b1d02979523..87f815d179d5 100644 --- a/processing/src/main/java/io/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/io/druid/segment/BitmapOffset.java @@ -36,7 +36,7 @@ /** */ -public class BitmapOffset implements Offset +public class BitmapOffset extends Offset { private static final int INVALID_VALUE = -1; private static final BitmapFactory ROARING_BITMAP_FACTORY = new RoaringBitmapSerdeFactory(false).getBitmapFactory(); diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index 23cfabf143a1..e11d4435c910 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -24,6 +24,7 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.spatial.ImmutableRTree; import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; @@ -99,6 +100,12 @@ public Iterator iterator() { return IndexedIterable.create(this).iterator(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", column); + } }; } diff --git a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java index e2a47e36eb5c..53cb38b1dde8 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java @@ -21,6 +21,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; import io.druid.segment.data.IndexedInts; @@ -65,11 +66,13 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier iterator() return IndexedIterable.create(this).iterator(); } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("offsets", offsets); + inspector.visit("values", values); + } } } diff --git a/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java b/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java index 4ade259a100e..3bf165becb4f 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java @@ -23,6 +23,7 @@ import com.google.common.base.Predicates; import io.druid.java.util.common.IAE; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.BooleanValueMatcher; @@ -81,6 +82,12 @@ public boolean matches() return false; } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } else { if (matchNull) { @@ -93,6 +100,12 @@ public boolean matches() final int size = row.size(); return size == 0; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } else { return BooleanValueMatcher.of(false); @@ -124,6 +137,12 @@ public boolean matches() return false; } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } @@ -169,6 +188,12 @@ public boolean matches() return false; } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } }; } @@ -197,6 +222,13 @@ public boolean matches() return false; } } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("predicate", predicate); + } }; } diff --git a/processing/src/main/java/io/druid/segment/FilteredOffset.java b/processing/src/main/java/io/druid/segment/FilteredOffset.java new file mode 100644 index 000000000000..8152e1a791c1 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/FilteredOffset.java @@ -0,0 +1,206 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.query.BaseQuery; +import io.druid.query.filter.BooleanFilter; +import io.druid.query.filter.Filter; +import io.druid.query.filter.RowOffsetMatcherFactory; +import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.data.Offset; +import io.druid.segment.filter.BooleanValueMatcher; +import io.druid.segment.historical.HistoricalCursor; +import io.druid.segment.historical.OffsetHolder; +import org.roaringbitmap.IntIterator; + +public final class FilteredOffset extends Offset +{ + private Offset baseOffset; + private final ValueMatcher filterMatcher; + + FilteredOffset( + HistoricalCursor cursor, + boolean descending, + Filter postFilter, + ColumnSelectorBitmapIndexSelector bitmapIndexSelector + ) + { + RowOffsetMatcherFactory rowOffsetMatcherFactory = new CursorOffsetHolderRowOffsetMatcherFactory( + cursor, + descending + ); + if (postFilter instanceof BooleanFilter) { + filterMatcher = ((BooleanFilter) postFilter).makeMatcher( + bitmapIndexSelector, + cursor, + rowOffsetMatcherFactory + ); + } else { + if (postFilter.supportsBitmapIndex(bitmapIndexSelector)) { + filterMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher( + postFilter.getBitmapIndex(bitmapIndexSelector) + ); + } else { + filterMatcher = postFilter.makeMatcher(cursor); + } + } + } + + void reset(Offset baseOffset) + { + this.baseOffset = baseOffset; + if (baseOffset.withinBounds()) { + if (!filterMatcher.matches()) { + BaseQuery.checkInterrupted(); + incrementInterruptibly(); + } + } + } + + @Override + public void increment() + { + baseOffset.increment(); + + while (baseOffset.withinBounds() && !Thread.currentThread().isInterrupted()) { + if (filterMatcher.matches()) { + return; + } else { + baseOffset.increment(); + } + } + } + + void incrementInterruptibly() + { + baseOffset.increment(); + while (baseOffset.withinBounds()) { + BaseQuery.checkInterrupted(); + if (filterMatcher.matches()) { + return; + } else { + baseOffset.increment(); + } + } + } + + @Override + public boolean withinBounds() + { + return baseOffset.withinBounds(); + } + + @Override + public Offset clone() + { + FilteredOffset offset = (FilteredOffset) super.clone(); + offset.baseOffset = offset.baseOffset.clone(); + return offset; + } + + @Override + public int getOffset() + { + return baseOffset.getOffset(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseOffset", baseOffset); + inspector.visit("filterMatcher", filterMatcher); + } + + private static class CursorOffsetHolderRowOffsetMatcherFactory implements RowOffsetMatcherFactory + { + private final OffsetHolder holder; + private final boolean descending; + + CursorOffsetHolderRowOffsetMatcherFactory(OffsetHolder holder, boolean descending) + { + this.holder = holder; + this.descending = descending; + } + + // Use an iterator-based implementation, ImmutableBitmap.get(index) works differently for Concise and Roaring. + // ImmutableConciseSet.get(index) is also inefficient, it performs a linear scan on each call + @Override + public ValueMatcher makeRowOffsetMatcher(final ImmutableBitmap rowBitmap) + { + final IntIterator iter = descending ? + BitmapOffset.getReverseBitmapOffsetIterator(rowBitmap) : + rowBitmap.iterator(); + + if (!iter.hasNext()) { + return BooleanValueMatcher.of(false); + } + + if (descending) { + return new ValueMatcher() + { + int iterOffset = Integer.MAX_VALUE; + + @Override + public boolean matches() + { + int currentOffset = holder.getOffset().getOffset(); + while (iterOffset > currentOffset && iter.hasNext()) { + iterOffset = iter.next(); + } + + return iterOffset == currentOffset; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("holder", holder); + inspector.visit("iter", iter); + } + }; + } else { + return new ValueMatcher() + { + int iterOffset = -1; + + @Override + public boolean matches() + { + int currentOffset = holder.getOffset().getOffset(); + while (iterOffset < currentOffset && iter.hasNext()) { + iterOffset = iter.next(); + } + + return iterOffset == currentOffset; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("holder", holder); + inspector.visit("iter", iter); + } + }; + } + } + } +} diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 36e0d2ce3ca5..a7f5f75450c5 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -120,6 +120,7 @@ public long get() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } @@ -150,6 +151,7 @@ public float get() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java index 7f4923630732..674e805d4548 100644 --- a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java @@ -77,6 +77,7 @@ public void close() throws IOException closeFile(outFile); } + @Override public void closeFile(final File outFile) throws IOException { outFile.delete(); diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index aae04e987188..c1b9d65f3055 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -94,6 +94,7 @@ import java.nio.ByteOrder; import java.util.AbstractList; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; @@ -605,14 +606,14 @@ public void convertV8toV9(File v8Dir, File v9Dir, IndexSpec indexSpec) ); bitmaps = GenericIndexed.fromIterable( - Iterables.concat(Arrays.asList(theNullSet), bitmaps), + Iterables.concat(Collections.singletonList(theNullSet), bitmaps), bitmapSerdeFactory.getObjectStrategy() ); } else { bumpedDictionary = false; bitmaps = GenericIndexed.fromIterable( Iterables.concat( - Arrays.asList( + Collections.singletonList( bitmapFactory .union(Arrays.asList(theNullSet, bitmaps.get(0))) ), diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 2b28a7e5409c..8c8e3be841b9 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.base.Splitter; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -44,13 +43,13 @@ import io.druid.common.guava.GuavaUtils; import io.druid.common.utils.JodaUtils; import io.druid.common.utils.SerializerUtils; -import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.guava.nary.BinaryFn; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; @@ -63,8 +62,6 @@ import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedIterable; -import io.druid.segment.data.ListIndexed; import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.TmpFileIOPeon; import io.druid.segment.incremental.IncrementalIndex; @@ -72,6 +69,9 @@ import io.druid.segment.serde.ComplexMetricColumnSerializer; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.ints.IntSortedSet; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -93,7 +93,6 @@ import java.util.NoSuchElementException; import java.util.PriorityQueue; import java.util.Set; -import java.util.TreeSet; /** */ @@ -101,10 +100,8 @@ public class IndexMerger { private static final Logger log = new Logger(IndexMerger.class); - protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Arrays.asList(""), String.class); protected static final SerializerUtils serializerUtils = new SerializerUtils(); protected static final int INVALID_ROW = -1; - protected static final Splitter SPLITTER = Splitter.on(","); protected final ObjectMapper mapper; protected final IndexIO indexIO; @@ -217,24 +214,8 @@ public File mergeQueryableIndex( ProgressIndicator progress ) throws IOException { - // We are materializing the list for performance reasons. Lists.transform - // only creates a "view" of the original list, meaning the function gets - // applied every time you access an element. - List indexAdapteres = Lists.newArrayList( - Iterables.transform( - indexes, - new Function() - { - @Override - public IndexableAdapter apply(final QueryableIndex input) - { - return new QueryableIndexIndexableAdapter(input); - } - } - ) - ); return merge( - indexAdapteres, + toIndexableAdapters(indexes), rollup, metricAggs, outDir, @@ -271,6 +252,26 @@ public Iterable apply(@Nullable IndexableAdapter input) ); } + private static List toIndexableAdapters(List indexes) + { + // We are materializing the list for performance reasons. Lists.transform + // only creates a "view" of the original list, meaning the function gets + // applied every time you access an element. + return Lists.newArrayList( + Iterables.transform( + indexes, + new Function() + { + @Override + public IndexableAdapter apply(final QueryableIndex input) + { + return new QueryableIndexIndexableAdapter(input); + } + } + ) + ); + } + private static List getLongestSharedDimOrder(List indexes) { int maxSize = 0; @@ -306,6 +307,11 @@ private static List getLongestSharedDimOrder(List inde return ImmutableList.copyOf(orderingCandidate); } + public static List getMergedDimensionsFromQueryableIndexes(List indexes) + { + return getMergedDimensions(toIndexableAdapters(indexes)); + } + public static List getMergedDimensions(List indexes) { if (indexes.size() == 0) { @@ -752,10 +758,14 @@ public Metadata apply(IndexableAdapter input) mergers.get(i).processMergedRow(dims[i]); } - for (Map.Entry> comprisedRow : theRow.getComprisedRows().entrySet()) { - final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey()); + Iterator> rowsIterator = theRow.getComprisedRows().int2ObjectEntrySet().fastIterator(); + while (rowsIterator.hasNext()) { + Int2ObjectMap.Entry comprisedRow = rowsIterator.next(); - for (Integer rowNum : comprisedRow.getValue()) { + final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getIntKey()); + + for (IntIterator setIterator = comprisedRow.getValue().iterator(); setIterator.hasNext(); /* NOP */) { + int rowNum = setIterator.nextInt(); while (conversionBuffer.position() < rowNum) { conversionBuffer.put(INVALID_ROW); } @@ -1074,6 +1084,7 @@ static class IndexSeekerWithConversion implements IndexSeeker this.lastVal = NOT_INIT; } + @Override public int seek(int dictId) { if (dimConversions == null) { @@ -1176,46 +1187,6 @@ public Rowboat apply(@Nullable Rowboat input) } } - public static class AggFactoryStringIndexed implements Indexed - { - private final AggregatorFactory[] metricAggs; - - public AggFactoryStringIndexed(AggregatorFactory[] metricAggs) - { - this.metricAggs = metricAggs; - } - - @Override - public Class getClazz() - { - return String.class; - } - - @Override - public int size() - { - return metricAggs.length; - } - - @Override - public String get(int index) - { - return metricAggs[index].getName(); - } - - @Override - public int indexOf(String value) - { - throw new UnsupportedOperationException(); - } - - @Override - public Iterator iterator() - { - return IndexedIterable.create(this).iterator(); - } - } - public static class RowboatMergeFunction implements BinaryFn { private final AggregatorFactory[] metricAggs; @@ -1260,9 +1231,12 @@ public Rowboat apply(Rowboat lhs, Rowboat rhs) ); for (Rowboat rowboat : Arrays.asList(lhs, rhs)) { - for (Map.Entry> entry : rowboat.getComprisedRows().entrySet()) { - for (Integer rowNum : entry.getValue()) { - retVal.addRow(entry.getKey(), rowNum); + Iterator> entryIterator = rowboat.getComprisedRows().int2ObjectEntrySet().fastIterator(); + while (entryIterator.hasNext()) { + Int2ObjectMap.Entry entry = entryIterator.next(); + + for (IntIterator setIterator = entry.getValue().iterator(); setIterator.hasNext(); /* NOP */) { + retVal.addRow(entry.getIntKey(), setIterator.nextInt()); } } } diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index eed32b9c72bc..81ca1123d99c 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -53,6 +53,9 @@ import io.druid.segment.serde.ComplexMetrics; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.ints.IntSortedSet; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -64,10 +67,10 @@ import java.nio.IntBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; public class IndexMergerV9 extends IndexMerger { @@ -452,16 +455,21 @@ private void mergeIndexesAndWriteColumns( merger.processMergedRow(dims[i]); } - for (Map.Entry> comprisedRow : theRow.getComprisedRows().entrySet()) { - final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey()); + Iterator> rowsIterator = theRow.getComprisedRows().int2ObjectEntrySet().fastIterator(); + while (rowsIterator.hasNext()) { + Int2ObjectMap.Entry comprisedRow = rowsIterator.next(); - for (Integer rowNum : comprisedRow.getValue()) { + final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getIntKey()); + + for (IntIterator setIterator = comprisedRow.getValue().iterator(); setIterator.hasNext(); /* NOP */) { + int rowNum = setIterator.nextInt(); while (conversionBuffer.position() < rowNum) { conversionBuffer.put(INVALID_ROW); } conversionBuffer.put(rowCount); } } + if ((++rowCount % 500000) == 0) { log.info("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time); time = System.currentTimeMillis(); diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 963c03383a9d..0b24837db1c8 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -119,6 +119,7 @@ public long get() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } @@ -150,6 +151,7 @@ public float get() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java index 263f740dcca0..8fe336ba1f54 100644 --- a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java @@ -81,6 +81,7 @@ public void close() throws IOException closeFile(outFile); } + @Override public void closeFile(final File outFile) throws IOException { outFile.delete(); diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 1782d8277361..0c3b82edc6ae 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -45,7 +45,6 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.WritableByteChannel; /** */ @@ -103,26 +102,6 @@ public static void writeLongMetric( column.closeAndConsolidate(outSupplier); } - public static void writeToChannel(MetricHolder holder, WritableByteChannel out) throws IOException - { - out.write(ByteBuffer.wrap(version)); - serializerUtils.writeString(out, holder.name); - serializerUtils.writeString(out, holder.typeName); - - switch (holder.type) { - case FLOAT: - holder.floatType.writeToChannel(out); - break; - case COMPLEX: - if (holder.complexType instanceof GenericIndexed) { - ((GenericIndexed) holder.complexType).writeToChannel(out); - } else { - throw new IAE("Cannot serialize out MetricHolder for complex type that is not a GenericIndexed"); - } - break; - } - } - public static MetricHolder fromByteBuffer(ByteBuffer buf, SmooshedFileMapper mapper) throws IOException { return fromByteBuffer(buf, null, mapper); diff --git a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java index 385bbad0c7d6..ef7fd5838ab2 100644 --- a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java @@ -26,10 +26,11 @@ import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ZeroIndexedInts; import io.druid.segment.filter.BooleanValueMatcher; +import io.druid.segment.historical.SingleValueHistoricalDimensionSelector; import javax.annotation.Nullable; -public class NullDimensionSelector implements DimensionSelector, IdLookup +public class NullDimensionSelector implements SingleValueHistoricalDimensionSelector, IdLookup { private static final NullDimensionSelector INSTANCE = new NullDimensionSelector(); @@ -49,6 +50,24 @@ public IndexedInts getRow() return ZeroIndexedInts.instance(); } + @Override + public int getRowValue() + { + return 0; + } + + @Override + public int getRowValue(int offset) + { + return 0; + } + + @Override + public IndexedInts getRow(int offset) + { + return getRow(); + } + @Override public ValueMatcher makeValueMatcher(String value) { @@ -96,5 +115,6 @@ public int lookupId(String name) @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndex.java b/processing/src/main/java/io/druid/segment/QueryableIndex.java index 31492b18cc43..52a5b3c17656 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndex.java @@ -43,5 +43,6 @@ public interface QueryableIndex extends ColumnSelector, Closeable * @throws java.io.IOException if an exception was thrown closing the index */ //@Deprecated // This is still required for SimpleQueryableIndex. It should not go away unitl SimpleQueryableIndex is fixed + @Override public void close() throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 7fdbf4e985a4..dad9a84b4ced 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.logger.Logger; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; @@ -165,6 +166,12 @@ public Iterator iterator() { return IndexedIterable.create(this).iterator(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("dict", dict); + } }; } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index d3af3b9e9e04..f94aa788c654 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -20,24 +20,18 @@ package io.druid.segment; import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.druid.collections.bitmap.ImmutableBitmap; -import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.io.Closer; import io.druid.query.BaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; -import io.druid.query.filter.BooleanFilter; import io.druid.query.filter.Filter; -import io.druid.query.filter.RowOffsetMatcherFactory; -import io.druid.query.filter.ValueMatcher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; @@ -50,19 +44,16 @@ import io.druid.segment.data.IndexedInts; import io.druid.segment.data.Offset; import io.druid.segment.filter.AndFilter; -import io.druid.segment.filter.BooleanValueMatcher; -import it.unimi.dsi.fastutil.ints.IntIterators; +import io.druid.segment.historical.HistoricalCursor; +import io.druid.segment.historical.HistoricalFloatColumnSelector; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.roaringbitmap.IntIterator; -import javax.annotation.Nullable; import java.io.Closeable; -import java.io.IOException; import java.util.ArrayList; -import java.util.BitSet; import java.util.List; import java.util.Map; +import java.util.Objects; /** */ @@ -305,7 +296,7 @@ public Sequence makeCursors( postFilter, selector ).build(), - Predicates.notNull() + Objects::nonNull ); } @@ -320,7 +311,6 @@ private static ColumnCapabilities getColumnCapabilites(ColumnSelector index, Str private static class CursorSequenceBuilder { - private final StorageAdapter storageAdapter; private final QueryableIndex index; private final Interval interval; private final VirtualColumns virtualColumns; @@ -345,7 +335,6 @@ public CursorSequenceBuilder( ColumnSelectorBitmapIndexSelector bitmapIndexSelector ) { - this.storageAdapter = storageAdapter; this.index = storageAdapter.index; this.interval = interval; this.virtualColumns = virtualColumns; @@ -418,11 +407,44 @@ public Cursor apply(final Interval inputInterval) final Offset initOffset = offset.clone(); final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis()); - final CursorOffsetHolder cursorOffsetHolder = new CursorOffsetHolder(); - abstract class QueryableIndexBaseCursor implements Cursor + abstract class QueryableIndexBaseCursor implements HistoricalCursor { - Offset cursorOffset; + OffsetType cursorOffset; + + @Override + public OffsetType getOffset() + { + return cursorOffset; + } + + @Override + public DateTime getTime() + { + return myBucket; + } + + @Override + public void advanceTo(int offset) + { + int count = 0; + while (count < offset && !isDone()) { + advance(); + count++; + } + } + + @Override + public boolean isDone() + { + return !cursorOffset.withinBounds(); + } + + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } @Override public DimensionSelector makeDimensionSelector( @@ -472,204 +494,13 @@ private DimensionSelector makeDimensionSelectorUndecorated( } final DictionaryEncodedColumn column = cachedColumn; - - abstract class QueryableDimensionSelector implements DimensionSelector, IdLookup - { - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", column); - inspector.visit("cursorOffset", cursorOffset); - inspector.visit("extractionFn", extractionFn); - } - } if (column == null) { return NullDimensionSelector.instance(); - } else if (columnDesc.getCapabilities().hasMultipleValues()) { - class MultiValueDimensionSelector extends QueryableDimensionSelector - { - @Override - public IndexedInts getRow() - { - return column.getMultiValueRow(cursorOffset.getOffset()); - } - - @Override - public ValueMatcher makeValueMatcher(String value) - { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, value); - } - - @Override - public ValueMatcher makeValueMatcher(Predicate predicate) - { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); - } - - @Override - public int getValueCardinality() - { - return column.getCardinality(); - } - - @Override - public String lookupName(int id) - { - final String value = column.lookupName(id); - return extractionFn == null ? - value : - extractionFn.apply(value); - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return true; - } - - @Nullable - @Override - public IdLookup idLookup() - { - return extractionFn == null ? this : null; - } - - @Override - public int lookupId(String name) - { - if (extractionFn != null) { - throw new UnsupportedOperationException( - "cannot perform lookup when applying an extraction function" - ); - } - return column.lookupId(name); - } - } - return new MultiValueDimensionSelector(); } else { - class SingleValueDimensionSelector extends QueryableDimensionSelector - { - @Override - public IndexedInts getRow() - { - // using an anonymous class is faster than creating a class that stores a copy of the value - return new IndexedInts() - { - @Override - public int size() - { - return 1; - } - - @Override - public int get(int index) - { - return column.getSingleValueRow(cursorOffset.getOffset()); - } - - @Override - public it.unimi.dsi.fastutil.ints.IntIterator iterator() - { - return IntIterators.singleton(column.getSingleValueRow(cursorOffset.getOffset())); - } - - @Override - public void fill(int index, int[] toFill) - { - throw new UnsupportedOperationException("fill not supported"); - } - - @Override - public void close() throws IOException - { - - } - }; - } - - @Override - public ValueMatcher makeValueMatcher(final String value) - { - if (extractionFn == null) { - final int valueId = lookupId(value); - if (valueId >= 0) { - return new ValueMatcher() - { - @Override - public boolean matches() - { - return column.getSingleValueRow(cursorOffset.getOffset()) == valueId; - } - }; - } else { - return BooleanValueMatcher.of(false); - } - } else { - // Employ precomputed BitSet optimization - return makeValueMatcher(Predicates.equalTo(value)); - } - } - - @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) - { - final BitSet predicateMatchingValueIds = DimensionSelectorUtils.makePredicateMatchingSet( - this, - predicate - ); - return new ValueMatcher() - { - @Override - public boolean matches() - { - int rowValueId = column.getSingleValueRow(cursorOffset.getOffset()); - return predicateMatchingValueIds.get(rowValueId); - } - }; - } - - @Override - public int getValueCardinality() - { - return column.getCardinality(); - } - - @Override - public String lookupName(int id) - { - final String value = column.lookupName(id); - return extractionFn == null ? value : extractionFn.apply(value); - } - - @Override - public boolean nameLookupPossibleInAdvance() - { - return true; - } - - @Nullable - @Override - public IdLookup idLookup() - { - return extractionFn == null ? this : null; - } - - @Override - public int lookupId(String name) - { - if (extractionFn != null) { - throw new UnsupportedOperationException( - "cannot perform lookup when applying an extraction function" - ); - } - return column.lookupId(name); - } - } - return new SingleValueDimensionSelector(); + return column.makeDimensionSelector(this, extractionFn); } } - @Override public FloatColumnSelector makeFloatColumnSelector(String columnName) { @@ -694,7 +525,7 @@ public FloatColumnSelector makeFloatColumnSelector(String columnName) } final GenericColumn metricVals = cachedMetricVals; - return new FloatColumnSelector() + return new HistoricalFloatColumnSelector() { @Override public float get() @@ -702,6 +533,12 @@ public float get() return metricVals.getFloatSingleValueRow(cursorOffset.getOffset()); } + @Override + public float get(int offset) + { + return metricVals.getFloatSingleValueRow(offset); + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { @@ -922,18 +759,12 @@ public ColumnCapabilities getColumnCapabilities(String columnName) } if (postFilter == null) { - return new QueryableIndexBaseCursor() + return new QueryableIndexBaseCursor() { { reset(); } - @Override - public DateTime getTime() - { - return myBucket; - } - @Override public void advance() { @@ -947,139 +778,39 @@ public void advanceUninterruptibly() cursorOffset.increment(); } - @Override - public void advanceTo(int offset) - { - int count = 0; - while (count < offset && !isDone()) { - advance(); - count++; - } - } - - @Override - public boolean isDone() - { - return !cursorOffset.withinBounds(); - } - - @Override - public boolean isDoneOrInterrupted() - { - return isDone() || Thread.currentThread().isInterrupted(); - } - @Override public void reset() { cursorOffset = initOffset.clone(); - cursorOffsetHolder.set(cursorOffset); } }; } else { - return new QueryableIndexBaseCursor() + return new QueryableIndexBaseCursor() { - RowOffsetMatcherFactory rowOffsetMatcherFactory = new CursorOffsetHolderRowOffsetMatcherFactory( - cursorOffsetHolder, - descending - ); - - final ValueMatcher filterMatcher; - - { - if (postFilter instanceof BooleanFilter) { - filterMatcher = ((BooleanFilter) postFilter).makeMatcher( - bitmapIndexSelector, - this, - rowOffsetMatcherFactory - ); - } else { - if (postFilter.supportsBitmapIndex(bitmapIndexSelector)) { - filterMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher(postFilter.getBitmapIndex( - bitmapIndexSelector)); - } else { - filterMatcher = postFilter.makeMatcher(this); - } - } - } - { + cursorOffset = new FilteredOffset(this, descending, postFilter, bitmapIndexSelector); reset(); } - @Override - public DateTime getTime() - { - return myBucket; - } - @Override public void advance() { BaseQuery.checkInterrupted(); - cursorOffset.increment(); - - while (!isDone()) { - BaseQuery.checkInterrupted(); - if (filterMatcher.matches()) { - return; - } else { - cursorOffset.increment(); - } - } + cursorOffset.incrementInterruptibly(); } @Override public void advanceUninterruptibly() { - if (Thread.currentThread().isInterrupted()) { - return; - } - cursorOffset.increment(); - - while (!isDoneOrInterrupted()) { - if (filterMatcher.matches()) { - return; - } else { - cursorOffset.increment(); - } - } - } - - @Override - public void advanceTo(int offset) - { - int count = 0; - while (count < offset && !isDone()) { - advance(); - count++; + if (!Thread.currentThread().isInterrupted()) { + cursorOffset.increment(); } } - @Override - public boolean isDone() - { - return !cursorOffset.withinBounds(); - } - - @Override - public boolean isDoneOrInterrupted() - { - return isDone() || Thread.currentThread().isInterrupted(); - } - @Override public void reset() { - cursorOffset = initOffset.clone(); - cursorOffsetHolder.set(cursorOffset); - if (!isDone()) { - if (filterMatcher.matches()) { - return; - } else { - advance(); - } - } + cursorOffset.reset(initOffset.clone()); } }; } @@ -1092,83 +823,7 @@ public void reset() } } - public static class CursorOffsetHolder - { - Offset currOffset = null; - - public Offset get() - { - return currOffset; - } - - public void set(Offset currOffset) - { - this.currOffset = currOffset; - } - } - - private static class CursorOffsetHolderRowOffsetMatcherFactory implements RowOffsetMatcherFactory - { - private final CursorOffsetHolder holder; - private final boolean descending; - - public CursorOffsetHolderRowOffsetMatcherFactory(CursorOffsetHolder holder, boolean descending) - { - this.holder = holder; - this.descending = descending; - } - - // Use an iterator-based implementation, ImmutableBitmap.get(index) works differently for Concise and Roaring. - // ImmutableConciseSet.get(index) is also inefficient, it performs a linear scan on each call - @Override - public ValueMatcher makeRowOffsetMatcher(final ImmutableBitmap rowBitmap) - { - final IntIterator iter = descending ? - BitmapOffset.getReverseBitmapOffsetIterator(rowBitmap) : - rowBitmap.iterator(); - - if (!iter.hasNext()) { - return BooleanValueMatcher.of(false); - } - - if (descending) { - return new ValueMatcher() - { - int iterOffset = Integer.MAX_VALUE; - - @Override - public boolean matches() - { - int currentOffset = holder.get().getOffset(); - while (iterOffset > currentOffset && iter.hasNext()) { - iterOffset = iter.next(); - } - - return iterOffset == currentOffset; - } - }; - } else { - return new ValueMatcher() - { - int iterOffset = -1; - - @Override - public boolean matches() - { - int currentOffset = holder.get().getOffset(); - while (iterOffset < currentOffset && iter.hasNext()) { - iterOffset = iter.next(); - } - - return iterOffset == currentOffset; - } - }; - } - } - } - - - private abstract static class TimestampCheckingOffset implements Offset + public abstract static class TimestampCheckingOffset extends Offset { protected final Offset baseOffset; protected final GenericColumn timestamps; @@ -1230,7 +885,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } } - private static class AscendingTimestampCheckingOffset extends TimestampCheckingOffset + public static class AscendingTimestampCheckingOffset extends TimestampCheckingOffset { public AscendingTimestampCheckingOffset( Offset baseOffset, @@ -1262,7 +917,7 @@ public Offset clone() } } - private static class DescendingTimestampCheckingOffset extends TimestampCheckingOffset + public static class DescendingTimestampCheckingOffset extends TimestampCheckingOffset { public DescendingTimestampCheckingOffset( Offset baseOffset, @@ -1295,7 +950,7 @@ public Offset clone() } } - private static class NoFilterOffset implements Offset + public static class NoFilterOffset extends Offset { private final int rowCount; private final boolean descending; diff --git a/processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java index 8a4505da3335..64c133370b55 100644 --- a/processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java @@ -181,6 +181,7 @@ private void innerClose() throws IOException } } + @Override public T as(Class clazz) { return getBaseSegment().as(clazz); diff --git a/processing/src/main/java/io/druid/segment/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java index a8ea20b7aac3..6d66669f75d5 100644 --- a/processing/src/main/java/io/druid/segment/Rowboat.java +++ b/processing/src/main/java/io/druid/segment/Rowboat.java @@ -19,15 +19,14 @@ package io.druid.segment; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.ints.IntRBTreeSet; +import it.unimi.dsi.fastutil.ints.IntSortedSet; import org.joda.time.DateTime; import java.util.Arrays; -import java.util.Map; -import java.util.TreeSet; public class Rowboat implements Comparable { @@ -35,7 +34,7 @@ public class Rowboat implements Comparable private final Object[] dims; private final Object[] metrics; private final int rowNum; - private final Map> comprisedRows; + private final Int2ObjectOpenHashMap comprisedRows; private final DimensionHandler[] handlers; public Rowboat( @@ -52,7 +51,7 @@ public Rowboat( this.rowNum = rowNum; this.handlers = handlers; - this.comprisedRows = Maps.newHashMap(); + this.comprisedRows = new Int2ObjectOpenHashMap<>(); } public long getTimestamp() @@ -72,15 +71,15 @@ public Object[] getMetrics() public void addRow(int indexNum, int rowNum) { - TreeSet rowNums = comprisedRows.get(indexNum); + IntSortedSet rowNums = comprisedRows.get(indexNum); if (rowNums == null) { - rowNums = Sets.newTreeSet(); + rowNums = new IntRBTreeSet(); comprisedRows.put(indexNum, rowNums); } rowNums.add(rowNum); } - public Map> getComprisedRows() + public Int2ObjectOpenHashMap getComprisedRows() { return comprisedRows; } diff --git a/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java b/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java index 9e4f296b2102..f7e41d572962 100644 --- a/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java +++ b/processing/src/main/java/io/druid/segment/SingleScanTimeDimSelector.java @@ -31,7 +31,7 @@ import java.util.List; import java.util.Objects; -public class SingleScanTimeDimSelector implements DimensionSelector +public class SingleScanTimeDimSelector implements SingleValueDimensionSelector { private final ExtractionFn extractionFn; private final LongColumnSelector selector; @@ -64,6 +64,12 @@ public IndexedInts getRow() return new SingleIndexedInt(getDimensionValueIndex()); } + @Override + public int getRowValue() + { + return getDimensionValueIndex(); + } + @Override public ValueMatcher makeValueMatcher(final String value) { @@ -74,6 +80,12 @@ public boolean matches() { return Objects.equals(lookupName(getDimensionValueIndex()), value); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", SingleScanTimeDimSelector.this); + } }; } @@ -87,6 +99,13 @@ public boolean matches() { return predicate.apply(lookupName(getDimensionValueIndex())); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", SingleScanTimeDimSelector.this); + inspector.visit("predicate", predicate); + } }; } diff --git a/processing/src/main/java/io/druid/segment/SingleValueDimensionSelector.java b/processing/src/main/java/io/druid/segment/SingleValueDimensionSelector.java new file mode 100644 index 000000000000..39f5730b78fb --- /dev/null +++ b/processing/src/main/java/io/druid/segment/SingleValueDimensionSelector.java @@ -0,0 +1,34 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; + +/** + * Specialization for {@link DimensionSelector}s, always having a single value in {@link #getRow()}. + */ +public interface SingleValueDimensionSelector extends DimensionSelector +{ + /** + * Returns a single value of {@link #getRow()}. + */ + @CalledFromHotLoop + int getRowValue(); +} diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 830547369adb..956a28a5feff 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -19,12 +19,12 @@ package io.druid.segment; -import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; @@ -43,6 +43,9 @@ import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import it.unimi.dsi.fastutil.ints.IntArrays; +import it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap; +import it.unimi.dsi.fastutil.objects.Object2IntSortedMap; import javax.annotation.Nullable; import java.util.ArrayList; @@ -52,38 +55,13 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.function.Function; public class StringDimensionIndexer implements DimensionIndexer { - public static final Function STRING_TRANSFORMER = new Function() - { - @Override - public String apply(final Object o) - { - if (o == null) { - return null; - } - if (o instanceof String) { - return (String) o; - } - return o.toString(); - } - }; + private static final Function STRING_TRANSFORMER = o -> o != null ? o.toString() : null; - public static final Comparator UNENCODED_COMPARATOR = new Comparator() - { - @Override - public int compare(String o1, String o2) - { - if (o1 == null) { - return o2 == null ? 0 : -1; - } - if (o2 == null) { - return 1; - } - return o1.compareTo(o2); - } - }; + private static final Comparator UNENCODED_COMPARATOR = Ordering.natural().nullsFirst(); private static class DimensionDictionary { @@ -176,7 +154,7 @@ private static class SortedDimensionDictionary public SortedDimensionDictionary(List idToValue, int length) { - Map sortedMap = Maps.newTreeMap(); + Object2IntSortedMap sortedMap = new Object2IntRBTreeMap<>(); for (int id = 0; id < length; id++) { sortedMap.put(idToValue.get(id), id); } @@ -184,7 +162,8 @@ public SortedDimensionDictionary(List idToValue, int length) this.idToIndex = new int[length]; this.indexToId = new int[length]; int index = 0; - for (Integer id : sortedMap.values()) { + for (IntIterator iterator = sortedMap.values().iterator(); iterator.hasNext();) { + int id = iterator.nextInt(); idToIndex[id] = index; indexToId[index] = id; index++; @@ -327,6 +306,12 @@ public Iterator iterator() { return IndexedIterable.create(this).iterator(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } }; } @@ -469,6 +454,12 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } }; } else { return BooleanValueMatcher.of(false); @@ -506,6 +497,12 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } }; } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 6e5fa0f43e92..2f8e23d6895a 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -30,13 +30,14 @@ import io.druid.collections.spatial.ImmutableRTree; import io.druid.collections.spatial.RTree; import io.druid.collections.spatial.split.LinearGutmanSplitStrategy; -import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.ByteBufferUtils; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.logger.Logger; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; +import io.druid.segment.data.ArrayIndexed; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedObjectStrategy; @@ -49,7 +50,6 @@ import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIntsWriter; import io.druid.segment.data.IndexedRTree; -import io.druid.segment.data.ListIndexed; import io.druid.segment.data.VSizeIndexedIntsWriter; import io.druid.segment.data.VSizeIndexedWriter; import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; @@ -64,14 +64,13 @@ import java.nio.IntBuffer; import java.nio.MappedByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; public class StringDimensionMergerV9 implements DimensionMergerV9 { private static final Logger log = new Logger(StringDimensionMergerV9.class); - protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Arrays.asList(""), String.class); + protected static final Indexed EMPTY_STR_DIM_VAL = new ArrayIndexed<>(new String[]{""}, String.class); protected static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0}; protected static final Splitter SPLITTER = Splitter.on(","); @@ -462,7 +461,7 @@ protected interface IndexSeeker int seek(int dictId); } - protected class IndexSeekerWithoutConversion implements IndexSeeker + protected static class IndexSeekerWithoutConversion implements IndexSeeker { private final int limit; @@ -481,7 +480,7 @@ public int seek(int dictId) /** * Get old dictId from new dictId, and only support access in order */ - protected class IndexSeekerWithConversion implements IndexSeeker + protected static class IndexSeekerWithConversion implements IndexSeeker { private final IntBuffer dimConversions; private int currIndex; @@ -496,6 +495,7 @@ protected class IndexSeekerWithConversion implements IndexSeeker this.lastVal = NOT_INIT; } + @Override public int seek(int dictId) { if (dimConversions == null) { diff --git a/processing/src/main/java/io/druid/segment/VirtualColumns.java b/processing/src/main/java/io/druid/segment/VirtualColumns.java index 9f4d97f60d5e..584a4308c371 100644 --- a/processing/src/main/java/io/druid/segment/VirtualColumns.java +++ b/processing/src/main/java/io/druid/segment/VirtualColumns.java @@ -226,6 +226,7 @@ public ColumnSelectorFactory wrap(final ColumnSelectorFactory baseFactory) return new VirtualizedColumnSelectorFactory(baseFactory, this); } + @Override public byte[] getCacheKey() { // id doesn't matter as there is only one kind of "VirtualColumns", so use 0. diff --git a/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java b/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java index 295ecd23539d..ffcd67e83fa6 100644 --- a/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ZeroFloatColumnSelector.java @@ -20,8 +20,9 @@ package io.druid.segment; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.historical.HistoricalFloatColumnSelector; -public final class ZeroFloatColumnSelector implements FloatColumnSelector +public final class ZeroFloatColumnSelector implements HistoricalFloatColumnSelector { private static final ZeroFloatColumnSelector INSTANCE = new ZeroFloatColumnSelector(); @@ -41,8 +42,15 @@ public float get() return 0.0f; } + @Override + public float get(int offset) + { + return get(); + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java b/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java index fbf765fd93e0..71cb8128ce1f 100644 --- a/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ZeroLongColumnSelector.java @@ -44,5 +44,6 @@ public long get() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java index 22d19b606f91..0a6cd57c8c2a 100644 --- a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -19,7 +19,10 @@ package io.druid.segment.column; +import io.druid.query.extraction.ExtractionFn; +import io.druid.segment.DimensionSelector; import io.druid.segment.data.IndexedInts; +import io.druid.segment.historical.OffsetHolder; import java.io.Closeable; @@ -34,4 +37,6 @@ public interface DictionaryEncodedColumn extends public ActualType lookupName(int id); public int lookupId(ActualType name); public int getCardinality(); + + DimensionSelector makeDimensionSelector(OffsetHolder offsetHolder, ExtractionFn extractionFn); } diff --git a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index 35fb8d03e4c0..754282cc2856 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -19,13 +19,27 @@ package io.druid.segment.column; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.base.Strings; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.DimensionSelectorUtils; +import io.druid.segment.IdLookup; import io.druid.segment.data.CachingIndexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedMultivalue; +import io.druid.segment.data.SingleIndexedInt; +import io.druid.segment.filter.BooleanValueMatcher; +import io.druid.segment.historical.HistoricalDimensionSelector; +import io.druid.segment.historical.OffsetHolder; +import io.druid.segment.historical.SingleValueHistoricalDimensionSelector; +import javax.annotation.Nullable; import java.io.IOException; +import java.util.BitSet; /** */ @@ -90,6 +104,184 @@ public int getCardinality() return cachedLookups.size(); } + @Override + public HistoricalDimensionSelector makeDimensionSelector( + final OffsetHolder offsetHolder, + final ExtractionFn extractionFn + ) + { + abstract class QueryableDimensionSelector implements HistoricalDimensionSelector, IdLookup + { + @Override + public int getValueCardinality() + { + return getCardinality(); + } + + @Override + public String lookupName(int id) + { + final String value = SimpleDictionaryEncodedColumn.this.lookupName(id); + return extractionFn == null ? + value : + extractionFn.apply(value); + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return true; + } + + @Nullable + @Override + public IdLookup idLookup() + { + return extractionFn == null ? this : null; + } + + @Override + public int lookupId(String name) + { + if (extractionFn != null) { + throw new UnsupportedOperationException( + "cannot perform lookup when applying an extraction function" + ); + } + return SimpleDictionaryEncodedColumn.this.lookupId(name); + } + } + + if (hasMultipleValues()) { + class MultiValueDimensionSelector extends QueryableDimensionSelector + { + @Override + public IndexedInts getRow() + { + return multiValueColumn.get(offsetHolder.getOffset().getOffset()); + } + + @Override + public IndexedInts getRow(int offset) + { + return multiValueColumn.get(offset); + } + + @Override + public ValueMatcher makeValueMatcher(String value) + { + return DimensionSelectorUtils.makeValueMatcherGeneric(this, value); + } + + @Override + public ValueMatcher makeValueMatcher(Predicate predicate) + { + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("multiValueColumn", multiValueColumn); + inspector.visit("offsetHolder", offsetHolder); + inspector.visit("offset", offsetHolder.getOffset()); + inspector.visit("extractionFn", extractionFn); + } + } + return new MultiValueDimensionSelector(); + } else { + class SingleValueQueryableDimensionSelector extends QueryableDimensionSelector + implements SingleValueHistoricalDimensionSelector + { + @Override + public IndexedInts getRow() + { + return new SingleIndexedInt(getRowValue()); + } + + @Override + public int getRowValue() + { + return column.get(offsetHolder.getOffset().getOffset()); + } + + @Override + public IndexedInts getRow(int offset) + { + return new SingleIndexedInt(getRowValue(offset)); + } + + @Override + public int getRowValue(int offset) + { + return column.get(offset); + } + + @Override + public ValueMatcher makeValueMatcher(final String value) + { + if (extractionFn == null) { + final int valueId = lookupId(value); + if (valueId >= 0) { + return new ValueMatcher() + { + @Override + public boolean matches() + { + return getRowValue() == valueId; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", SimpleDictionaryEncodedColumn.this); + } + }; + } else { + return BooleanValueMatcher.of(false); + } + } else { + // Employ precomputed BitSet optimization + return makeValueMatcher(Predicates.equalTo(value)); + } + } + + @Override + public ValueMatcher makeValueMatcher(final Predicate predicate) + { + final BitSet predicateMatchingValueIds = DimensionSelectorUtils.makePredicateMatchingSet( + this, + predicate + ); + return new ValueMatcher() + { + @Override + public boolean matches() + { + return predicateMatchingValueIds.get(getRowValue()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", SimpleDictionaryEncodedColumn.this); + } + }; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", column); + inspector.visit("offsetHolder", offsetHolder); + inspector.visit("offset", offsetHolder.getOffset()); + inspector.visit("extractionFn", extractionFn); + } + } + return new SingleValueQueryableDimensionSelector(); + } + } + @Override public void close() throws IOException { diff --git a/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java index 71ad4025d0aa..f78ba2fca54a 100644 --- a/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import io.druid.java.util.common.IAE; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.ints.IntArrays; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterators; @@ -90,6 +91,11 @@ public void fill(int index, int[] toFill) @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java b/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java index 41385ec77c7d..90ce1da5dbee 100644 --- a/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java @@ -23,7 +23,7 @@ /** */ -public class ArrayBasedOffset implements Offset +public class ArrayBasedOffset extends Offset { private final int[] ints; private int currIndex; @@ -73,5 +73,6 @@ public Offset clone() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/data/ArrayIndexed.java b/processing/src/main/java/io/druid/segment/data/ArrayIndexed.java index 81a594e2f562..5cbc4bcf0a8e 100644 --- a/processing/src/main/java/io/druid/segment/data/ArrayIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayIndexed.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + import java.util.Arrays; import java.util.Iterator; @@ -67,4 +69,10 @@ public Iterator iterator() { return Arrays.asList(baseArray).iterator(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } } diff --git a/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java index 2c78d39429cf..a1f6346dedc0 100644 --- a/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/BitmapCompressedIndexedInts.java @@ -21,6 +21,7 @@ import com.google.common.collect.Ordering; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.IntIteratorUtils; import it.unimi.dsi.fastutil.ints.IntIterator; @@ -96,6 +97,11 @@ public void fill(int index, int[] toFill) @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("immutableBitmap", immutableBitmap); } } diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java index 729d75cc9a26..2976758f9a1a 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java @@ -87,6 +87,7 @@ public long get(int index) return longBuffer.get(longBuffer.position() + bufferIndex); } + @Override protected void loadBuffer(int bufferNum) { CloseQuietly.close(holder); diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java index 49da3f312a42..abdae5da7a97 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java @@ -23,7 +23,6 @@ import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; import com.google.common.primitives.Ints; - import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; import io.druid.java.util.common.io.smoosh.FileSmoosher; @@ -62,7 +61,7 @@ public BlockLayoutLongSupplierSerializer( { this.ioPeon = ioPeon; this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE); - this.flattener = new GenericIndexedWriter>( + this.flattener = new GenericIndexedWriter<>( ioPeon, filenameBase, VSizeCompressedObjectStrategy.getBufferForOrder( diff --git a/processing/src/main/java/io/druid/segment/data/CachingIndexed.java b/processing/src/main/java/io/druid/segment/data/CachingIndexed.java index 16b941718439..3f8e68fa780f 100644 --- a/processing/src/main/java/io/druid/segment/data/CachingIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/CachingIndexed.java @@ -21,6 +21,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.io.Closeable; import java.io.IOException; @@ -73,7 +74,7 @@ public int size() @Override public T get(int index) { - if(cachedValues != null) { + if (cachedValues != null) { final T cached = cachedValues.getValue(index); if (cached != null) { return cached; @@ -108,7 +109,14 @@ public void close() throws IOException } } -private static class SizedLRUMap extends LinkedHashMap> + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("cachedValues", cachedValues != null); + inspector.visit("delegate", delegate); + } + + private static class SizedLRUMap extends LinkedHashMap> { private final int maxBytes; private int numBytes = 0; diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java index 4e56126354b0..3978dcb8ce8c 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.CompressedPools; import it.unimi.dsi.fastutil.ints.IntIterator; @@ -94,6 +95,7 @@ public int get(int index) } } + @Override public long getSerializedSize() { return 1 + // version @@ -103,6 +105,7 @@ public long getSerializedSize() baseIntBuffers.getSerializedSize(); // data } + @Override public void writeToChannel(WritableByteChannel channel) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{VERSION})); @@ -365,5 +368,14 @@ public void close() throws IOException { Closeables.close(holder, false); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // ideally should inspect buffer, but at the moment of inspectRuntimeShape() call buffer is likely to be null, + // because loadBuffer() is not yet called, although during the processing it is not null, hence "visiting" null is + // not representative. + inspector.visit("singleThreadedIntBuffers", singleThreadedIntBuffers); + } } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java index 4ea7e3de8ccd..8151dc4e27ab 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java @@ -75,14 +75,14 @@ public CompressedIntsIndexedWriter( public CompressedIntsIndexedWriter( final int chunkFactor, final CompressedObjectStrategy.CompressionStrategy compression, - GenericIndexedWriter> flattner + GenericIndexedWriter> flattener ) { this.chunkFactor = chunkFactor; this.compression = compression; this.endBuffer = IntBuffer.allocate(chunkFactor); this.numInserted = 0; - this.flattener = flattner; + this.flattener = flattener; } @Override diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java index dd678c44c581..b18318c0c7f9 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.io.Closeables; import com.google.common.primitives.Ints; @@ -28,6 +29,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.CompressedPools; import it.unimi.dsi.fastutil.ints.IntIterator; @@ -122,6 +124,7 @@ public IndexedInts get() } + @Override public long getSerializedSize() { return 1 + // version @@ -132,6 +135,7 @@ public long getSerializedSize() baseBuffers.getSerializedSize(); // data } + @Override public void writeToChannel(WritableByteChannel channel) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes})); @@ -141,9 +145,7 @@ public void writeToChannel(WritableByteChannel channel) throws IOException baseBuffers.writeToChannel(channel); } - /** - * For testing. Do not use unless you like things breaking - */ + @VisibleForTesting GenericIndexed> getBaseBuffers() { return baseBuffers; @@ -357,7 +359,7 @@ public int get(int index) /** * Returns the value at the given index in the current decompression buffer * - * @param index index of the value in the curent buffer + * @param index index of the value in the current buffer * * @return the value at the given index */ @@ -409,5 +411,14 @@ public void close() throws IOException { Closeables.close(holder, false); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // ideally should inspect buffer and bigEndian, but at the moment of inspectRuntimeShape() call buffer is likely + // to be null and bigEndian = false, because loadBuffer() is not yet called, although during the processing buffer + // is not null, hence "visiting" null is not representative, and visiting bigEndian = false could be misleading. + inspector.visit("singleThreadedBuffers", singleThreadedBuffers); + } } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java index 09c4da117e22..94b2335762f7 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -63,7 +63,7 @@ private CompressionFactory() // encoding format for segments created prior to the introduction of encoding formats public static final LongEncodingFormat LEGACY_LONG_ENCODING_FORMAT = LongEncodingFormat.LONGS; - /* + /** * Delta Encoding Header v1: * Byte 1 : version * Byte 2 - 9 : base value @@ -71,7 +71,7 @@ private CompressionFactory() */ public static final byte DELTA_ENCODING_VERSION = 0x1; - /* + /** * Table Encoding Header v1 : * Byte 1 : version * Byte 2 - 5 : table size @@ -112,7 +112,7 @@ public static byte clearEncodingFlag(byte strategyId) return hasEncodingFlag(strategyId) ? (byte) (strategyId + FLAG_VALUE) : strategyId; } - /* + /** * The compression of decompression of encodings are separated into different enums. EncodingStrategy refers to the * strategy used to encode the data, and EncodingFormat refers to the format the data is encoded in. Note there is not * necessarily an one-to-one mapping between to two. For instance, the AUTO LongEncodingStrategy scans the data once @@ -120,7 +120,6 @@ public static byte clearEncodingFlag(byte strategyId) * write in any of the LongEncodingFormat. On the other hand, there are no LongEncodingStrategy that always write in * TABLE LongEncodingFormat since it only works for data with low cardinality. */ - public enum LongEncodingStrategy { /** diff --git a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java index 922fa5e8fd47..e73bd72f4722 100644 --- a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java @@ -57,6 +57,7 @@ public void write(long value) throws IOException serializer.write(value - base); } + @Override public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException { metaOut.write(CompressionFactory.setEncodingFlag(strategy.getId())); diff --git a/processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java b/processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java index 7b604b950aa6..870808ce5eb3 100644 --- a/processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterators; @@ -61,6 +62,11 @@ public void fill(int index, int[] toFill) @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index d67f6df0d2c8..61f3395049cb 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.bytes.ByteArrays; import java.io.Closeable; @@ -166,7 +167,7 @@ static int getNumberOfFilesRequired(int bagSize, long numWritten) private int logBaseTwoOfElementsPerValueFile; private int relativeIndexMask; - private ByteBuffer theBuffer; + private final ByteBuffer theBuffer; /** * Constructor for version one. @@ -210,6 +211,7 @@ static int getNumberOfFilesRequired(int bagSize, long numWritten) { this.versionOne = false; + this.theBuffer = null; this.strategy = strategy; this.allowReverseLookup = allowReverseLookup; this.valueBuffers = valueBuffs; @@ -347,6 +349,21 @@ private T copyBufferAndGet(ByteBuffer valueBuffer, int startOffset, int endOffse return strategy.fromByteBuffer(copyValueBuffer, size); } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("versionOne", versionOne); + inspector.visit("headerBuffer", headerBuffer); + if (versionOne) { + inspector.visit("firstValueBuffer", firstValueBuffer); + } else { + // Inspecting just one example of valueBuffer, not needed to inspect the whole array, because all buffers in it + // are the same. + inspector.visit("valueBuffer", valueBuffers.length > 0 ? valueBuffers[0] : null); + } + inspector.visit("strategy", strategy); + } + abstract class BufferIndexed implements Indexed { int lastReadSize; @@ -525,6 +542,14 @@ public T get(final int index) } return bufferedIndexedGet(copyBuffer, startOffset, endOffset); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("headerBuffer", headerBuffer); + inspector.visit("copyBuffer", copyBuffer); + inspector.visit("strategy", strategy); + } }; } @@ -633,6 +658,16 @@ public T get(final int index) int fileNum = index >> logBaseTwoOfElementsPerValueFile; return bufferedIndexedGet(copyValueBuffers[fileNum], startOffset, endOffset); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("headerBuffer", headerBuffer); + // Inspecting just one example of copyValueBuffer, not needed to inspect the whole array, because all buffers + // in it are the same. + inspector.visit("copyValueBuffer", copyValueBuffers.length > 0 ? copyValueBuffers[0] : null); + inspector.visit("strategy", strategy); + } }; } } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index d2e91f094f34..621ea05ac6b2 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -391,7 +391,6 @@ private void writeToChannelVersionTwo(WritableByteChannel channel, FileSmoosher } writeHeaderLong(smoosher, headerFile, bagSizePower, buffer); } - } public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException @@ -403,7 +402,8 @@ public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) t } } - private void writeHeaderLong(FileSmoosher smoosher, RandomAccessFile headerFile, int bagSizePower, byte[] buffer) throws IOException + private void writeHeaderLong(FileSmoosher smoosher, RandomAccessFile headerFile, int bagSizePower, byte[] buffer) + throws IOException { ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES).order(ByteOrder.nativeOrder()); diff --git a/processing/src/main/java/io/druid/segment/data/Indexed.java b/processing/src/main/java/io/druid/segment/data/Indexed.java index d64ccf81f3fd..1f6ee2ebf1ca 100644 --- a/processing/src/main/java/io/druid/segment/data/Indexed.java +++ b/processing/src/main/java/io/druid/segment/data/Indexed.java @@ -19,12 +19,16 @@ package io.druid.segment.data; -public interface Indexed extends Iterable +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; + +public interface Indexed extends Iterable, HotLoopCallee { Class getClazz(); int size(); + @CalledFromHotLoop T get(int index); /** diff --git a/processing/src/main/java/io/druid/segment/data/IndexedInts.java b/processing/src/main/java/io/druid/segment/data/IndexedInts.java index 2e097b24618c..63af05f01699 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedInts.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.HotLoopCallee; import it.unimi.dsi.fastutil.ints.IntIterable; import java.io.Closeable; @@ -26,9 +28,11 @@ /** * Get a int an index (array or list lookup abstraction without boxing). */ -public interface IndexedInts extends IntIterable, Closeable +public interface IndexedInts extends IntIterable, Closeable, HotLoopCallee { + @CalledFromHotLoop int size(); + @CalledFromHotLoop int get(int index); void fill(int index, int[] toFill); } diff --git a/processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java b/processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java index ccc362fc0e7f..267c7fde50cb 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java @@ -30,7 +30,7 @@ public class IndexedIntsIterator extends AbstractIntIterator private final IndexedInts baseInts; private final int size; - int currIndex = 0; + private int currIndex = 0; public IndexedIntsIterator( IndexedInts baseInts @@ -48,7 +48,8 @@ public boolean hasNext() } @Override - public int nextInt() { + public int nextInt() + { return baseInts.get(currIndex++); } diff --git a/processing/src/main/java/io/druid/segment/data/IndexedMultivalue.java b/processing/src/main/java/io/druid/segment/data/IndexedMultivalue.java index 1303396e7134..1c3906ee148a 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedMultivalue.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedMultivalue.java @@ -23,4 +23,6 @@ public interface IndexedMultivalue extends Indexed, Closeable { + @Override + T get(int index); } diff --git a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java index 6b459f489164..c775e4f3de20 100644 --- a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java @@ -75,16 +75,19 @@ public IntermediateLongSupplierSerializer( this.compression = compression; } + @Override public void open() throws IOException { tempOut = new CountingOutputStream(ioPeon.makeOutputStream(tempFile)); } + @Override public int size() { return numInserted; } + @Override public void add(long value) throws IOException { SerializerUtils.writeBigEndianLongToOutputStream(tempOut, value, helperBuffer); @@ -139,6 +142,7 @@ private void makeDelegate() throws IOException } } + @Override public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException { tempOut.close(); @@ -146,6 +150,7 @@ public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException delegate.closeAndConsolidate(consolidatedOut); } + @Override public void close() throws IOException { tempOut.close(); @@ -153,11 +158,13 @@ public void close() throws IOException delegate.close(); } + @Override public long getSerializedSize() { return delegate.getSerializedSize(); } + @Override public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { delegate.writeToChannel(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java b/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java index f3da84d0a1e1..94e4cdc3792b 100644 --- a/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java +++ b/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java @@ -23,7 +23,7 @@ /** */ -public class IntersectingOffset implements Offset { +public class IntersectingOffset extends Offset { private final Offset lhs; private final Offset rhs; diff --git a/processing/src/main/java/io/druid/segment/data/ListIndexed.java b/processing/src/main/java/io/druid/segment/data/ListIndexed.java index feffc0d63a1c..b06b7ef7c1ec 100644 --- a/processing/src/main/java/io/druid/segment/data/ListIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/ListIndexed.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; + import java.util.Iterator; import java.util.List; @@ -67,4 +69,10 @@ public Iterator iterator() { return baseList.iterator(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseList", baseList); + } } diff --git a/processing/src/main/java/io/druid/segment/data/Offset.java b/processing/src/main/java/io/druid/segment/data/Offset.java index 7e44dd7b8e7f..7eff391e65d8 100644 --- a/processing/src/main/java/io/druid/segment/data/Offset.java +++ b/processing/src/main/java/io/druid/segment/data/Offset.java @@ -19,19 +19,35 @@ package io.druid.segment.data; +import io.druid.annotations.SubclassesMustBePublic; import io.druid.query.monomorphicprocessing.CalledFromHotLoop; /** * The "mutable" version of a ReadableOffset. Introduces "increment()" and "withinBounds()" methods, which are * very similar to "next()" and "hasNext()" on the Iterator interface except increment() does not return a value. + * + * Annotated with {@link SubclassesMustBePublic} because Offset occurrences are replaced with a subclass in {@link + * io.druid.query.topn.Historical1SimpleDoubleAggPooledTopNScannerPrototype} and {@link + * io.druid.query.topn.HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype} during + * specialization, and specialized version of those prototypes must be able to any subclass of Offset. */ -public interface Offset extends ReadableOffset +@SubclassesMustBePublic +public abstract class Offset implements ReadableOffset, Cloneable { @CalledFromHotLoop - void increment(); + public abstract void increment(); @CalledFromHotLoop - boolean withinBounds(); + public abstract boolean withinBounds(); - Offset clone(); + @Override + public Offset clone() + { + try { + return (Offset) super.clone(); + } + catch (CloneNotSupportedException e) { + throw new RuntimeException(e); + } + } } diff --git a/processing/src/main/java/io/druid/segment/data/RangeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/RangeIndexedInts.java index 172c74e307f7..0f426f32ec73 100644 --- a/processing/src/main/java/io/druid/segment/data/RangeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/RangeIndexedInts.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import com.google.common.base.Preconditions; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterators; @@ -86,6 +87,11 @@ public IntIterator iterator() @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/data/SingleIndexedInt.java b/processing/src/main/java/io/druid/segment/data/SingleIndexedInt.java index 2799cf9b31a7..0d7eaac4b80f 100644 --- a/processing/src/main/java/io/druid/segment/data/SingleIndexedInt.java +++ b/processing/src/main/java/io/druid/segment/data/SingleIndexedInt.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterators; @@ -64,4 +65,10 @@ public void fill(int index, int[] toFill) public void close() throws IOException { } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } } diff --git a/processing/src/main/java/io/druid/segment/data/UnioningOffset.java b/processing/src/main/java/io/druid/segment/data/UnioningOffset.java index 05be50016b5e..225a18961f1c 100644 --- a/processing/src/main/java/io/druid/segment/data/UnioningOffset.java +++ b/processing/src/main/java/io/druid/segment/data/UnioningOffset.java @@ -23,7 +23,7 @@ /** */ -public class UnioningOffset implements Offset +public class UnioningOffset extends Offset { private final Offset[] offsets = new Offset[2]; private final int[] offsetVals = new int[2]; diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java index 873c56752169..1f7eb057b746 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java @@ -24,6 +24,7 @@ import io.druid.io.ZeroCopyByteArrayOutputStream; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.io.IOException; import java.nio.ByteBuffer; @@ -190,6 +191,12 @@ public void close() throws IOException // no-op } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("theBuffer", theBuffer); + } + public WritableSupplier> asWritableSupplier() { return new VSizeIndexedSupplier(this); } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java index 95b72122aeb9..a35c58813f50 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import com.google.common.primitives.Ints; import io.druid.java.util.common.IAE; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.ints.IntIterator; import java.io.IOException; @@ -220,7 +221,12 @@ public void fill(int index, int[] toFill) @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("buffer", buffer); } public WritableSupplier asWritableSupplier() { diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java index 37ea293acd16..fb8670a613c0 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java @@ -70,6 +70,7 @@ public VSizeIndexedWriter( this.maxId = maxId; } + @Override public void open() throws IOException { headerOut = new CountingOutputStream(ioPeon.makeOutputStream(headerFileName)); @@ -116,16 +117,11 @@ public void close() throws IOException numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten ); - OutputStream metaOut = ioPeon.makeOutputStream(metaFileName); - - try { + try (OutputStream metaOut = ioPeon.makeOutputStream(metaFileName)) { metaOut.write(new byte[]{VERSION, numBytesForMax}); metaOut.write(Ints.toByteArray((int) numBytesWritten + 4)); metaOut.write(Ints.toByteArray(numWritten)); } - finally { - metaOut.close(); - } } public InputSupplier combineStreams() @@ -133,8 +129,8 @@ public InputSupplier combineStreams() return ByteStreams.join( Iterables.transform( Arrays.asList(metaFileName, headerFileName, valuesFileName), - new Function>() { - + new Function>() + { @Override public InputSupplier apply(final String input) { diff --git a/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java b/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java index 2ebcf666aab6..295c48cbab49 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java @@ -497,6 +497,7 @@ public Size16Des(ByteBuffer buffer, int bufferOffset) this.offset = bufferOffset; } + @Override public long get(int index) { return buffer.getShort(offset + (index << 1)) & 0xFFFF; diff --git a/processing/src/main/java/io/druid/segment/data/ZeroIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ZeroIndexedInts.java index 1ee254da7235..a26e215d8a28 100644 --- a/processing/src/main/java/io/druid/segment/data/ZeroIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/ZeroIndexedInts.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterators; @@ -70,6 +71,11 @@ public IntIterator iterator() @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect } } diff --git a/processing/src/main/java/io/druid/segment/filter/AndFilter.java b/processing/src/main/java/io/druid/segment/filter/AndFilter.java index dbe4c7f45f09..5b993b263e6b 100644 --- a/processing/src/main/java/io/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/AndFilter.java @@ -28,6 +28,7 @@ import io.druid.query.filter.Filter; import io.druid.query.filter.RowOffsetMatcherFactory; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelector; import io.druid.segment.ColumnSelectorFactory; @@ -39,13 +40,13 @@ public class AndFilter implements BooleanFilter { private static final Joiner AND_JOINER = Joiner.on(" && "); + static final ValueMatcher[] EMPTY_VALUE_MATCHER_ARRAY = new ValueMatcher[0]; private final List filters; - public AndFilter( - List filters - ) + public AndFilter(List filters) { + Preconditions.checkArgument(filters.size() > 0, "Can't construct empty AndFilter"); this.filters = filters; } @@ -80,10 +81,6 @@ public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector) @Override public ValueMatcher makeMatcher(ColumnSelectorFactory factory) { - if (filters.size() == 0) { - return BooleanValueMatcher.of(false); - } - final ValueMatcher[] matchers = new ValueMatcher[filters.size()]; for (int i = 0; i < filters.size(); i++) { @@ -117,19 +114,7 @@ public ValueMatcher makeMatcher( matchers.add(0, offsetMatcher); } - return new ValueMatcher() - { - @Override - public boolean matches() - { - for (ValueMatcher valueMatcher : matchers) { - if (!valueMatcher.matches()) { - return false; - } - } - return true; - } - }; + return makeMatcher(matchers.toArray(EMPTY_VALUE_MATCHER_ARRAY)); } @Override @@ -181,6 +166,7 @@ public String toString() private ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) { + Preconditions.checkState(baseMatchers.length > 0); if (baseMatchers.length == 1) { return baseMatchers[0]; } @@ -197,6 +183,15 @@ public boolean matches() } return true; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("firstBaseMatcher", baseMatchers[0]); + inspector.visit("secondBaseMatcher", baseMatchers[1]); + // Don't inspect the 3rd and all consequent baseMatchers, cut runtime shape combinations at this point. + // Anyway if the filter is so complex, Hotspot won't inline all calls because of the inline limit. + } }; } diff --git a/processing/src/main/java/io/druid/segment/filter/ColumnComparisonFilter.java b/processing/src/main/java/io/druid/segment/filter/ColumnComparisonFilter.java index f8e126a2424d..f69fca8b9eab 100644 --- a/processing/src/main/java/io/druid/segment/filter/ColumnComparisonFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/ColumnComparisonFilter.java @@ -29,11 +29,13 @@ import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherColumnSelectorStrategy; import io.druid.query.filter.ValueMatcherColumnSelectorStrategyFactory; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionHandlerUtils; import java.util.List; +import java.util.Objects; /** */ @@ -74,6 +76,9 @@ public ValueMatcher makeMatcher(ColumnSelectorFactory factory) } public static ValueMatcher makeValueMatcher(final ValueGetter[] valueGetters) { + if (valueGetters.length == 0) { + return BooleanValueMatcher.of(true); + } return new ValueMatcher() { @Override @@ -93,6 +98,13 @@ public boolean matches() } return true; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // All value getters are likely the same or similar (in terms of runtime shape), so inspecting only one of them. + inspector.visit("oneValueGetter", valueGetters[0]); + } }; } @@ -109,11 +121,7 @@ public static boolean overlap(String[] a, String[] b) { for (int i = 0; i < a.length; i++) { for (int j = 0; j < b.length; j++) { - if (a[i] == null || b[j] == null) { - if (a[i] == b[j]) { - return true; - } - } else if (a[i].equals(b[j])) { + if (Objects.equals(a[i], b[j])) { return true; } } diff --git a/processing/src/main/java/io/druid/segment/filter/FalseValueMatcher.java b/processing/src/main/java/io/druid/segment/filter/FalseValueMatcher.java index 189dae6a6332..2467dd79dcbe 100644 --- a/processing/src/main/java/io/druid/segment/filter/FalseValueMatcher.java +++ b/processing/src/main/java/io/druid/segment/filter/FalseValueMatcher.java @@ -20,6 +20,7 @@ package io.druid.segment.filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; final class FalseValueMatcher implements ValueMatcher { @@ -39,4 +40,10 @@ public boolean matches() { return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } } diff --git a/processing/src/main/java/io/druid/segment/filter/Filters.java b/processing/src/main/java/io/druid/segment/filter/Filters.java index 756bcee52e17..a8e8a34a74a1 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -38,6 +38,7 @@ import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherColumnSelectorStrategy; import io.druid.query.filter.ValueMatcherColumnSelectorStrategyFactory; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionHandlerUtils; @@ -453,6 +454,13 @@ public boolean matches() { return predicate.applyLong(longSelector.get()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("longSelector", longSelector); + inspector.visit("predicate", predicate); + } }; } diff --git a/processing/src/main/java/io/druid/segment/filter/NotFilter.java b/processing/src/main/java/io/druid/segment/filter/NotFilter.java index cbba340b5c86..35a365bc63ef 100644 --- a/processing/src/main/java/io/druid/segment/filter/NotFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/NotFilter.java @@ -23,6 +23,7 @@ import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelector; import io.druid.segment.ColumnSelectorFactory; @@ -60,6 +61,12 @@ public boolean matches() { return !baseMatcher.matches(); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("baseMatcher", baseMatcher); + } }; } diff --git a/processing/src/main/java/io/druid/segment/filter/OrFilter.java b/processing/src/main/java/io/druid/segment/filter/OrFilter.java index 02fe68f937f4..58a99f5331e0 100644 --- a/processing/src/main/java/io/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/OrFilter.java @@ -20,6 +20,7 @@ package io.druid.segment.filter; import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.filter.BitmapIndexSelector; @@ -27,6 +28,7 @@ import io.druid.query.filter.Filter; import io.druid.query.filter.RowOffsetMatcherFactory; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelector; import io.druid.segment.ColumnSelectorFactory; @@ -41,13 +43,9 @@ public class OrFilter implements BooleanFilter private final List filters; - public OrFilter( - List filters - ) + public OrFilter(List filters) { - if (filters.size() == 0) { - throw new IllegalArgumentException("Can't construct empty OrFilter (the universe does not exist)"); - } + Preconditions.checkArgument(filters.size() > 0, "Can't construct empty OrFilter (the universe does not exist)"); this.filters = filters; } @@ -103,23 +101,13 @@ public ValueMatcher makeMatcher( matchers.add(0, offsetMatcher); } - return new ValueMatcher() - { - @Override - public boolean matches() - { - for (ValueMatcher valueMatcher : matchers) { - if (valueMatcher.matches()) { - return true; - } - } - return false; - } - }; + return makeMatcher(matchers.toArray(AndFilter.EMPTY_VALUE_MATCHER_ARRAY)); } private ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers){ + Preconditions.checkState(baseMatchers.length > 0); + if (baseMatchers.length == 1) { return baseMatchers[0]; } @@ -136,6 +124,15 @@ public boolean matches() } return false; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("firstBaseMatcher", baseMatchers[0]); + inspector.visit("secondBaseMatcher", baseMatchers[1]); + // Don't inspect the 3rd and all consequent baseMatchers, cut runtime shape combinations at this point. + // Anyway if the filter is so complex, Hotspot won't inline all calls because of the inline limit. + } }; } @@ -180,6 +177,7 @@ public double estimateSelectivity(BitmapIndexSelector indexSelector) return Math.min(selectivity, 1.); } + @Override public String toString() { return String.format("(%s)", OR_JOINER.join(filters)); diff --git a/processing/src/main/java/io/druid/segment/filter/TrueValueMatcher.java b/processing/src/main/java/io/druid/segment/filter/TrueValueMatcher.java index 288e43870824..bfcdb7a32131 100644 --- a/processing/src/main/java/io/druid/segment/filter/TrueValueMatcher.java +++ b/processing/src/main/java/io/druid/segment/filter/TrueValueMatcher.java @@ -20,6 +20,7 @@ package io.druid.segment.filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; final class TrueValueMatcher implements ValueMatcher { @@ -39,4 +40,10 @@ public boolean matches() { return true; } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } } diff --git a/processing/src/main/java/io/druid/segment/historical/HistoricalCursor.java b/processing/src/main/java/io/druid/segment/historical/HistoricalCursor.java new file mode 100644 index 000000000000..ce289ce27d0c --- /dev/null +++ b/processing/src/main/java/io/druid/segment/historical/HistoricalCursor.java @@ -0,0 +1,26 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.historical; + +import io.druid.segment.Cursor; + +public interface HistoricalCursor extends Cursor, OffsetHolder +{ +} diff --git a/processing/src/main/java/io/druid/segment/historical/HistoricalDimensionSelector.java b/processing/src/main/java/io/druid/segment/historical/HistoricalDimensionSelector.java new file mode 100644 index 000000000000..698ed2f4d454 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/historical/HistoricalDimensionSelector.java @@ -0,0 +1,33 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.historical; + +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.segment.DimensionSelector; +import io.druid.segment.data.IndexedInts; + +/** + * Specialization for {@link DimensionSelector} queryable via offsets from {@link HistoricalCursor}. + */ +public interface HistoricalDimensionSelector extends DimensionSelector +{ + @CalledFromHotLoop + IndexedInts getRow(int offset); +} diff --git a/processing/src/main/java/io/druid/segment/historical/HistoricalFloatColumnSelector.java b/processing/src/main/java/io/druid/segment/historical/HistoricalFloatColumnSelector.java new file mode 100644 index 000000000000..d2b91dab76ad --- /dev/null +++ b/processing/src/main/java/io/druid/segment/historical/HistoricalFloatColumnSelector.java @@ -0,0 +1,29 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.historical; + +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.segment.FloatColumnSelector; + +public interface HistoricalFloatColumnSelector extends FloatColumnSelector +{ + @CalledFromHotLoop + float get(int offset); +} diff --git a/processing/src/main/java/io/druid/segment/historical/OffsetHolder.java b/processing/src/main/java/io/druid/segment/historical/OffsetHolder.java new file mode 100644 index 000000000000..438617d7207f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/historical/OffsetHolder.java @@ -0,0 +1,27 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.historical; + +import io.druid.segment.data.Offset; + +public interface OffsetHolder +{ + Offset getOffset(); +} diff --git a/processing/src/main/java/io/druid/segment/historical/SingleValueHistoricalDimensionSelector.java b/processing/src/main/java/io/druid/segment/historical/SingleValueHistoricalDimensionSelector.java new file mode 100644 index 000000000000..f5f6ba29d65a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/historical/SingleValueHistoricalDimensionSelector.java @@ -0,0 +1,30 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.historical; + +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.segment.SingleValueDimensionSelector; + +public interface SingleValueHistoricalDimensionSelector + extends HistoricalDimensionSelector, SingleValueDimensionSelector +{ + @CalledFromHotLoop + int getRowValue(int offset); +} diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 1daa24ad61e1..194d04f2e812 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -25,7 +25,6 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -37,9 +36,9 @@ import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.SpatialDimensionSchema; -import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.granularity.Granularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -70,6 +69,7 @@ import java.lang.reflect.Array; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.Deque; import java.util.Iterator; @@ -204,14 +204,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) // This is modified on add() in a critical section. private final ThreadLocal in = new ThreadLocal<>(); - private final Supplier rowSupplier = new Supplier() - { - @Override - public InputRow get() - { - return in.get(); - } - }; + private final Supplier rowSupplier = in::get; /** * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that @@ -275,7 +268,7 @@ public IncrementalIndex( columnCapabilities.put(dimName, capabilities); } - //__time capabilites + //__time capabilities ColumnCapabilitiesImpl timeCapabilities = new ColumnCapabilitiesImpl(); timeCapabilities.setType(ValueType.LONG); columnCapabilities.put(Column.TIME_COLUMN_NAME, timeCapabilities); @@ -595,7 +588,7 @@ private ColumnCapabilitiesImpl makeCapabilitesFromValueType(ValueType type) return capabilities; } - /* + /** * Currently called to initialize IncrementalIndex dimension order during index creation * Index dimension ordering could be changed to initialize from DimensionsSpec after resolution of * https://github.com/druid-io/druid/issues/2011 @@ -687,47 +680,41 @@ public Iterator iterator() return Iterators.transform( getFacts().iterator(descending), - new Function, Row>() - { - @Override - public Row apply(final Map.Entry input) - { - final TimeAndDims timeAndDims = input.getKey(); - final int rowOffset = input.getValue(); - - Object[] theDims = timeAndDims.getDims(); - - Map theVals = Maps.newLinkedHashMap(); - for (int i = 0; i < theDims.length; ++i) { - Object dim = theDims[i]; - DimensionDesc dimensionDesc = dimensions.get(i); - if (dimensionDesc == null) { - continue; - } - String dimensionName = dimensionDesc.getName(); - DimensionHandler handler = dimensionDesc.getHandler(); - if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) { - theVals.put(dimensionName, null); - continue; - } - final DimensionIndexer indexer = dimensionDesc.getIndexer(); - Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualArrayOrList(dim, DimensionIndexer.LIST); - theVals.put(dimensionName, rowVals); - } + timeAndDims -> { + final int rowOffset = timeAndDims.getRowIndex(); - AggregatorType[] aggs = getAggsForRow(rowOffset); - for (int i = 0; i < aggs.length; ++i) { - theVals.put(metrics[i].getName(), getAggVal(aggs[i], rowOffset, i)); - } + Object[] theDims = timeAndDims.getDims(); - if (postAggs != null) { - for (PostAggregator postAgg : postAggs) { - theVals.put(postAgg.getName(), postAgg.compute(theVals)); - } + Map theVals = Maps.newLinkedHashMap(); + for (int i = 0; i < theDims.length; ++i) { + Object dim = theDims[i]; + DimensionDesc dimensionDesc = dimensions.get(i); + if (dimensionDesc == null) { + continue; + } + String dimensionName = dimensionDesc.getName(); + DimensionHandler handler = dimensionDesc.getHandler(); + if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) { + theVals.put(dimensionName, null); + continue; } + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualArrayOrList(dim, DimensionIndexer.LIST); + theVals.put(dimensionName, rowVals); + } - return new MapBasedRow(timeAndDims.getTimestamp(), theVals); + AggregatorType[] aggs = getAggsForRow(rowOffset); + for (int i = 0; i < aggs.length; ++i) { + theVals.put(metrics[i].getName(), getAggVal(aggs[i], rowOffset, i)); } + + if (postAggs != null) { + for (PostAggregator postAgg : postAggs) { + theVals.put(postAgg.getName(), postAgg.compute(theVals)); + } + } + + return new MapBasedRow(timeAndDims.getTimestamp(), theVals); } ); } @@ -831,19 +818,40 @@ public ColumnCapabilitiesImpl getCapabilities() public static final class TimeAndDims { + public static final int EMPTY_ROW_INDEX = -1; + private final long timestamp; private final Object[] dims; private final List dimensionDescsList; + /** + * rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. TimeAndDims acts as a Map key + * and "entry" object (rowIndex is the "value") at the same time. This is done to reduce object indirection and + * improve locality, and avoid boxing of rowIndex as Integer, when stored in JDK collection: + * {@link RollupFactsHolder} needs concurrent collections, that are not present in fastutil. + */ + private int rowIndex; + TimeAndDims( long timestamp, Object[] dims, List dimensionDescsList ) + { + this(timestamp, dims, dimensionDescsList, EMPTY_ROW_INDEX); + } + + TimeAndDims( + long timestamp, + Object[] dims, + List dimensionDescsList, + int rowIndex + ) { this.timestamp = timestamp; this.dims = dims; this.dimensionDescsList = dimensionDescsList; + this.rowIndex = rowIndex; } public long getTimestamp() @@ -856,6 +864,16 @@ public Object[] getDims() return dims; } + public int getRowIndex() + { + return rowIndex; + } + + private void setRowIndex(int rowIndex) + { + this.rowIndex = rowIndex; + } + @Override public String toString() { @@ -868,9 +886,9 @@ public String toString() public Object apply(@Nullable Object input) { if (input == null || Array.getLength(input) == 0) { - return Arrays.asList("null"); + return Collections.singletonList("null"); } - return Arrays.asList(input); + return Collections.singletonList(input); } } ) + '}'; @@ -990,85 +1008,29 @@ private static boolean allNull(Object[] dims, int startPosition) return true; } - public static class FactsEntry implements Map.Entry - { - TimeAndDims key = null; - Integer value = null; - - public FactsEntry(TimeAndDims key, Integer value) - { - this.key = key; - this.value = value; - } - - public TimeAndDims getKey() - { - return key; - } - - public Integer getValue() - { - return value; - } - - @Override - public Integer setValue(Integer value) - { - return value; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - FactsEntry that = (FactsEntry) o; - - if (key != null ? !key.equals(that.key) : that.key != null) { - return false; - } - return value != null ? value.equals(that.value) : that.value == null; - } - - @Override - public int hashCode() - { - int result = key != null ? key.hashCode() : 0; - result = 31 * result + (value != null ? value.hashCode() : 0); - return result; - } - } - interface FactsHolder { /** - * @return the previous value associated with the specified key, or - * {@code null} if there was no mapping for the key. + * @return the previous rowIndex associated with the specified key, or + * {@code TimeAndDims#EMPTY_ROW_INDEX} if there was no mapping for the key. */ - Integer getPriorIndex(TimeAndDims key); + int getPriorIndex(TimeAndDims key); long getMinTimeMillis(); long getMaxTimeMillis(); - Iterable> entrySet(); + Iterator iterator(boolean descending); - Iterator> iterator(boolean descending); - - Iterable> timeRangeIterable(boolean descending, long timeStart, long timeEnd); + Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); Iterable keySet(); /** - * @return the previous value associated with the specified key, or - * {@code null} if there was no mapping for the key. + * @return the previous rowIndex associated with the specified key, or + * {@code TimeAndDims#EMPTY_ROW_INDEX} if there was no mapping for the key. */ - Integer putIfAbsent(TimeAndDims key, Integer rowIndex); + int putIfAbsent(TimeAndDims key, int rowIndex); void clear(); } @@ -1076,7 +1038,8 @@ interface FactsHolder static class RollupFactsHolder implements FactsHolder { private final boolean sortFacts; - private final ConcurrentMap facts; + // Can't use Set because we need to be able to get from collection + private final ConcurrentMap facts; private final List dimensionDescsList; public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsComparator, List dimensionDescsList) @@ -1091,16 +1054,17 @@ public RollupFactsHolder(boolean sortFacts, Comparator timeAndDimsC } @Override - public Integer getPriorIndex(TimeAndDims key) + public int getPriorIndex(TimeAndDims key) { - return facts.get(key); + TimeAndDims timeAndDims = facts.get(key); + return timeAndDims == null ? TimeAndDims.EMPTY_ROW_INDEX : timeAndDims.rowIndex; } @Override public long getMinTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap) facts).firstKey().getTimestamp(); + return ((ConcurrentNavigableMap) facts).firstKey().getTimestamp(); } else { throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); } @@ -1110,38 +1074,33 @@ public long getMinTimeMillis() public long getMaxTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap) facts).lastKey().getTimestamp(); + return ((ConcurrentNavigableMap) facts).lastKey().getTimestamp(); } else { throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); } } - public Iterable> entrySet() - { - return facts.entrySet(); - } - @Override - public Iterator> iterator(boolean descending) + public Iterator iterator(boolean descending) { if (descending && sortFacts) { - return ((ConcurrentNavigableMap) facts).descendingMap().entrySet().iterator(); + return ((ConcurrentNavigableMap) facts).descendingMap().keySet().iterator(); } - return entrySet().iterator(); + return keySet().iterator(); } @Override - public Iterable> timeRangeIterable(boolean descending, long timeStart, long timeEnd) + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) { if (!sortFacts) { throw new UnsupportedOperationException("can't get timeRange from unsorted facts data."); } TimeAndDims start = new TimeAndDims(timeStart, new Object[]{}, dimensionDescsList); TimeAndDims end = new TimeAndDims(timeEnd, new Object[]{}, dimensionDescsList); - ConcurrentNavigableMap subMap = - ((ConcurrentNavigableMap) facts).subMap(start, end); - final Map rangeMap = descending ? subMap.descendingMap() : subMap; - return rangeMap.entrySet(); + ConcurrentNavigableMap subMap = + ((ConcurrentNavigableMap) facts).subMap(start, end); + final Map rangeMap = descending ? subMap.descendingMap() : subMap; + return rangeMap.keySet(); } @Override @@ -1151,9 +1110,12 @@ public Iterable keySet() } @Override - public Integer putIfAbsent(TimeAndDims key, Integer rowIndex) + public int putIfAbsent(TimeAndDims key, int rowIndex) { - return facts.putIfAbsent(key, rowIndex); + // setRowIndex() must be called before facts.putIfAbsent() for visibility of rowIndex from concurrent readers. + key.setRowIndex(rowIndex); + TimeAndDims prev = facts.putIfAbsent(key, key); + return prev == null ? TimeAndDims.EMPTY_ROW_INDEX : prev.rowIndex; } @Override @@ -1166,37 +1128,30 @@ public void clear() static class PlainFactsHolder implements FactsHolder { private final boolean sortFacts; - private final ConcurrentMap>> facts; + private final ConcurrentMap> facts; public PlainFactsHolder(boolean sortFacts) { this.sortFacts = sortFacts; if (sortFacts) { - this.facts = new ConcurrentSkipListMap<>(new Comparator() - { - @Override - public int compare(Long lhs, Long rhs) - { - return Longs.compare(lhs, rhs); - } - }); + this.facts = new ConcurrentSkipListMap<>(); } else { this.facts = new ConcurrentHashMap<>(); } } @Override - public Integer getPriorIndex(TimeAndDims key) + public int getPriorIndex(TimeAndDims key) { - // always return null to indicate that no prior key cause we always add new row - return null; + // always return EMPTY_ROW_INDEX to indicate that no prior key cause we always add new row + return TimeAndDims.EMPTY_ROW_INDEX; } @Override public long getMinTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap>>) facts).firstKey(); + return ((ConcurrentNavigableMap>) facts).firstKey(); } else { throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); } @@ -1206,92 +1161,65 @@ public long getMinTimeMillis() public long getMaxTimeMillis() { if (sortFacts) { - return ((ConcurrentNavigableMap>>) facts).lastKey(); + return ((ConcurrentNavigableMap>) facts).lastKey(); } else { throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); } } - public Iterable> entrySet() - { - return concat(facts.values(), false); - } - @Override - public Iterator> iterator(boolean descending) + public Iterator iterator(boolean descending) { if (descending && sortFacts) { - return concat(((ConcurrentNavigableMap>>) facts) + return concat(((ConcurrentNavigableMap>) facts) .descendingMap().values(), true).iterator(); } return concat(facts.values(), false).iterator(); } @Override - public Iterable> timeRangeIterable(boolean descending, long timeStart, long timeEnd) + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) { - ConcurrentNavigableMap>> subMap = - ((ConcurrentNavigableMap>>) facts).subMap(timeStart, timeEnd); - final Map>> rangeMap = descending ? subMap.descendingMap() : subMap; + ConcurrentNavigableMap> subMap = + ((ConcurrentNavigableMap>) facts).subMap(timeStart, timeEnd); + final Map> rangeMap = descending ? subMap.descendingMap() : subMap; return concat(rangeMap.values(), descending); } - private Iterable> concat( - final Iterable>> iterable, + private Iterable concat( + final Iterable> iterable, final boolean descending ) { - return new Iterable>() - { - @Override - public Iterator> iterator() - { - return Iterators.concat( - Iterators.transform( - iterable.iterator(), - new Function>, Iterator>>() - { - @Override - public Iterator> apply(Deque> input) - { - return descending ? input.descendingIterator() : input.iterator(); - } - } - ) - ); - } - }; + return () -> Iterators.concat( + Iterators.transform( + iterable.iterator(), + input -> descending ? input.descendingIterator() : input.iterator() + ) + ); } @Override public Iterable keySet() { - return Iterables.transform( - entrySet(), - new Function, TimeAndDims>() - { - @Override - public TimeAndDims apply(Map.Entry input) - { - return input.getKey(); - } - } - ); + return concat(facts.values(), false); } @Override - public Integer putIfAbsent(TimeAndDims key, Integer rowIndex) + public int putIfAbsent(TimeAndDims key, int rowIndex) { Long time = key.getTimestamp(); - Deque> rows = facts.get(time); + Deque rows = facts.get(time); if (rows == null) { - facts.putIfAbsent(time, new ConcurrentLinkedDeque>()); + facts.putIfAbsent(time, new ConcurrentLinkedDeque<>()); // in race condition, rows may be put by other thread, so always get latest status from facts rows = facts.get(time); } - rows.add(new FactsEntry(key, rowIndex)); - // always return null to indicate that we always add new row - return null; + // setRowIndex() must be called before rows.add() for visibility of rowIndex from concurrent readers. + key.setRowIndex(rowIndex); + rows.add(key); + // always return EMPTY_ROW_INDEX to indicate that we always add new row + return TimeAndDims.EMPTY_ROW_INDEX; } @Override diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index 72cd2707049f..caefaf7ce26f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -25,6 +25,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; import io.druid.java.util.common.logger.Logger; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionHandler; import io.druid.segment.DimensionIndexer; import io.druid.segment.IndexableAdapter; @@ -53,7 +54,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter private final IncrementalIndex index; private final Map accessors; - private class DimensionAccessor + private static class DimensionAccessor { private final IncrementalIndex.DimensionDesc dimensionDesc; private final MutableBitmap[] invertedIndexes; @@ -178,17 +179,16 @@ public Iterator iterator() * iterator() call to ensure the counter starts at 0. */ return Iterators.transform( - index.getFacts().entrySet().iterator(), - new Function, Rowboat>() + index.getFacts().keySet().iterator(), + new Function() { int count = 0; @Override - public Rowboat apply(Map.Entry input) + public Rowboat apply(IncrementalIndex.TimeAndDims timeAndDims) { - final IncrementalIndex.TimeAndDims timeAndDims = input.getKey(); final Object[] dimValues = timeAndDims.getDims(); - final int rowOffset = input.getValue(); + final int rowOffset = timeAndDims.getRowIndex(); Object[] dims = new Object[dimValues.length]; for (IncrementalIndex.DimensionDesc dimension : dimensions) { @@ -301,6 +301,12 @@ public void fill(int index, int[] toFill) public void close() throws IOException { } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("bitmapIndex", bitmapIndex); + } } @Override diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 2768d1047592..aa194d8c00eb 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -241,8 +241,8 @@ public Cursor apply(@Nullable final Interval interval) { private final ValueMatcher filterMatcher = makeFilterMatcher(filter, this); private final int maxRowIndex; - private Iterator> baseIter; - private Iterable> cursorIterable; + private Iterator baseIter; + private Iterable cursorIterable; private boolean emptyRange; final DateTime time; int numAdvanced = -1; @@ -278,8 +278,8 @@ public void advance() while (baseIter.hasNext()) { BaseQuery.checkInterrupted(); - Map.Entry entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getValue())) { + IncrementalIndex.TimeAndDims entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { continue; } @@ -306,8 +306,8 @@ public void advanceUninterruptibly() return; } - Map.Entry entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getValue())) { + IncrementalIndex.TimeAndDims entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { continue; } @@ -358,8 +358,8 @@ public void reset() boolean foundMatched = false; while (baseIter.hasNext()) { - Map.Entry entry = baseIter.next(); - if (beyondMaxRowIndex(entry.getValue())) { + IncrementalIndex.TimeAndDims entry = baseIter.next(); + if (beyondMaxRowIndex(entry.getRowIndex())) { numAdvanced++; continue; } @@ -490,6 +490,7 @@ public long get() @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // nothing to inspect } } return new TimeLongColumnSelector(); @@ -641,26 +642,26 @@ private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor) public static class EntryHolder { - Map.Entry currEntry = null; + IncrementalIndex.TimeAndDims currEntry = null; - public Map.Entry get() + public IncrementalIndex.TimeAndDims get() { return currEntry; } - public void set(Map.Entry currEntry) + public void set(IncrementalIndex.TimeAndDims currEntry) { this.currEntry = currEntry; } public IncrementalIndex.TimeAndDims getKey() { - return currEntry.getKey(); + return currEntry; } - public Integer getValue() + public int getValue() { - return currEntry.getValue(); + return currEntry.getRowIndex(); } } diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 059cc41283d0..d01f9969fb3b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -189,8 +189,8 @@ protected Integer addToFacts( int bufferOffset; synchronized (this) { - final Integer priorIndex = facts.getPriorIndex(key); - if (null != priorIndex) { + final int priorIndex = facts.getPriorIndex(key); + if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) { final int[] indexAndOffset = indexAndOffsets.get(priorIndex); bufferIndex = indexAndOffset[0]; bufferOffset = indexAndOffset[1]; @@ -202,9 +202,7 @@ protected Integer addToFacts( rowContainer.set(row); for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; - getAggs()[i] = agg.factorizeBuffered( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) - ); + getAggs()[i] = agg.factorizeBuffered(selectors.get(agg.getName())); } rowContainer.set(null); } @@ -236,7 +234,7 @@ protected Integer addToFacts( } // Last ditch sanity checks - if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == null) { + if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount); } @@ -245,8 +243,8 @@ protected Integer addToFacts( // note that indexAndOffsets must be updated before facts, because as soon as we update facts // concurrent readers get hold of it and might ask for newly added row indexAndOffsets.add(new int[]{bufferIndex, bufferOffset}); - final Integer prev = facts.putIfAbsent(key, rowIndex); - if (null == prev) { + final int prev = facts.putIfAbsent(key, rowIndex); + if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { throw new ISE("WTF! we are in sychronized block."); diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 82d572c36d5d..8afec755ee50 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -180,11 +180,11 @@ protected Integer addToFacts( Supplier rowSupplier ) throws IndexSizeExceededException { - final Integer priorIndex = facts.getPriorIndex(key); + final int priorIndex = facts.getPriorIndex(key); Aggregator[] aggs; - if (null != priorIndex) { + if (TimeAndDims.EMPTY_ROW_INDEX != priorIndex) { aggs = concurrentGet(priorIndex); doAggregate(metrics, aggs, rowContainer, row, reportParseExceptions); } else { @@ -196,11 +196,11 @@ protected Integer addToFacts( concurrentSet(rowIndex, aggs); // Last ditch sanity checks - if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == null) { + if (numEntries.get() >= maxRowCount && facts.getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount); } - final Integer prev = facts.putIfAbsent(key, rowIndex); - if (null == prev) { + final int prev = facts.putIfAbsent(key, rowIndex); + if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { // We lost a race diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java index d53715f488b6..dcdc9d0ddaae 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java @@ -36,7 +36,7 @@ import io.druid.java.util.common.parsers.ParseException; import org.joda.time.DateTime; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -189,7 +189,7 @@ public int compareTo(Row o) } if (spatialDimVals.size() == spatialDim.getDims().size()) { - spatialLookup.put(spatialDimName, Arrays.asList(JOINER.join(spatialDimVals))); + spatialLookup.put(spatialDimName, Collections.singletonList(JOINER.join(spatialDimVals))); finalDims.add(spatialDimName); } } diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 67375f9539a4..a306357b44dc 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -447,7 +447,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo final WritableSupplier> rMultiValuedColumn; if (hasMultipleValues) { - rMultiValuedColumn = readMultiValuedColum(rVersion, buffer, rFlags, builder.getFileMapper()); + rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags, builder.getFileMapper()); rSingleValuedColumn = null; } else { rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer, builder.getFileMapper()); @@ -495,12 +495,13 @@ private WritableSupplier readSingleValuedColumn( case UNCOMPRESSED_SINGLE_VALUE: return VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier(); case COMPRESSED: - return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder, fileMapper); + return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder, fileMapper); + default: + throw new IAE("Unsupported single-value version[%s]", version); } - throw new IAE("Unsupported single-value version[%s]", version); } - private WritableSupplier> readMultiValuedColum( + private WritableSupplier> readMultiValuedColumn( VERSION version, ByteBuffer buffer, int flags, SmooshedFileMapper fileMapper ) { @@ -515,8 +516,9 @@ private WritableSupplier> readMultiValuedColum( } else { throw new IAE("Unrecognized multi-value flag[%d]", flags); } + default: + throw new IAE("Unsupported multi-value version[%s]", version); } - throw new IAE("Unsupported multi-value version[%s]", version); } }; } diff --git a/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java b/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java index 1207c275c9f8..d7425cd08c0c 100644 --- a/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/BaseSingleValueDimensionSelector.java @@ -22,15 +22,17 @@ import com.google.common.base.Predicate; import io.druid.query.filter.ValueMatcher; import io.druid.query.monomorphicprocessing.CalledFromHotLoop; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.DimensionSelector; import io.druid.segment.IdLookup; +import io.druid.segment.SingleValueDimensionSelector; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.ZeroIndexedInts; import javax.annotation.Nullable; import java.util.Objects; -public abstract class BaseSingleValueDimensionSelector implements DimensionSelector +public abstract class BaseSingleValueDimensionSelector implements SingleValueDimensionSelector { @CalledFromHotLoop protected abstract String getValue(); @@ -41,6 +43,12 @@ public IndexedInts getRow() return ZeroIndexedInts.instance(); } + @Override + public int getRowValue() + { + return 0; + } + @Override public int getValueCardinality() { @@ -63,6 +71,12 @@ public boolean matches() { return Objects.equals(getValue(), value); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", BaseSingleValueDimensionSelector.this); + } }; } @@ -76,6 +90,13 @@ public boolean matches() { return predicate.apply(getValue()); } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", BaseSingleValueDimensionSelector.this); + inspector.visit("predicate", predicate); + } }; } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java index 9db9f9ddab45..757f7deddef7 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java @@ -130,6 +130,7 @@ protected String getValue() public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("baseSelector", baseSelector); + inspector.visit("extractionFn", extractionFn); } } return new ExtractionExpressionDimensionSelector(); diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java deleted file mode 100644 index 23da291170fd..000000000000 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.data.input; - -import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.TimeAndDimsParseSpec; -import io.druid.data.input.impl.TimestampSpec; -import org.joda.time.DateTime; -import org.junit.Test; - -import java.io.ByteArrayOutputStream; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.List; - -import static org.junit.Assert.assertEquals; - -public class ProtoBufInputRowParserTest -{ - - public static final String[] DIMENSIONS = new String[]{"eventType", "id", "someOtherId", "isValid"}; - - /* - * eventType = 1; - * - * required uint64 id = 2; - * required string timestamp = 3; - * optional uint32 someOtherId = 4; - * optional bool isValid = 5; - * optional string description = 6; - * - * optional float someFloatColumn = 7; - * optional uint32 someIntColumn = 8; - * optional uint64 someLongColumn = 9; - */ - - @Test - public void testParse() throws Exception - { - - //configure parser with desc file - ProtoBufInputRowParser parser = new ProtoBufInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)), Arrays.asList(), null) - ), - "prototest.desc" - ); - - - //create binary of proto test event - DateTime dateTime = new DateTime(2012, 07, 12, 9, 30); - ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder() - .setDescription("description") - .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) - .setId(4711L) - .setIsValid(true) - .setSomeOtherId(4712) - .setTimestamp(dateTime.toString()) - .setSomeFloatColumn(47.11F) - .setSomeIntColumn(815) - .setSomeLongColumn(816L) - .build(); - - ByteArrayOutputStream out = new ByteArrayOutputStream(); - event.writeTo(out); - - InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray())); - System.out.println(row); - - assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions()); - assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); - - assertDimensionEquals(row, "id", "4711"); - assertDimensionEquals(row, "isValid", "true"); - assertDimensionEquals(row, "someOtherId", "4712"); - assertDimensionEquals(row, "description", "description"); - assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name()); - - - assertEquals(47.11F, row.getFloatMetric("someFloatColumn"), 0.0); - assertEquals(815.0F, row.getFloatMetric("someIntColumn"), 0.0); - assertEquals(816.0F, row.getFloatMetric("someLongColumn"), 0.0); - - } - - private void assertDimensionEquals(InputRow row, String dimension, Object expected) - { - List values = row.getDimension(dimension); - assertEquals(1, values.size()); - assertEquals(expected, values.get(0)); - } -} diff --git a/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java b/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java deleted file mode 100644 index d18b28517b4d..000000000000 --- a/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java +++ /dev/null @@ -1,1068 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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. - */ - -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: ProtoTest.proto - -package io.druid.data.input; - -import com.google.protobuf.AbstractMessage; -import com.google.protobuf.UnknownFieldSet; - -public final class ProtoTestEventWrapper { - private ProtoTestEventWrapper() {} - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - } - public interface ProtoTestEventOrBuilder - extends com.google.protobuf.MessageOrBuilder { - - // required .prototest.ProtoTestEvent.EventCategory eventType = 1; - boolean hasEventType(); - ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType(); - - // required uint64 id = 2; - boolean hasId(); - long getId(); - - // required string timestamp = 3; - boolean hasTimestamp(); - String getTimestamp(); - - // optional uint32 someOtherId = 4; - boolean hasSomeOtherId(); - int getSomeOtherId(); - - // optional bool isValid = 5; - boolean hasIsValid(); - boolean getIsValid(); - - // optional string description = 6; - boolean hasDescription(); - String getDescription(); - - // optional float someFloatColumn = 7; - boolean hasSomeFloatColumn(); - float getSomeFloatColumn(); - - // optional uint32 someIntColumn = 8; - boolean hasSomeIntColumn(); - int getSomeIntColumn(); - - // optional uint64 someLongColumn = 9; - boolean hasSomeLongColumn(); - long getSomeLongColumn(); - } - public static final class ProtoTestEvent extends - com.google.protobuf.GeneratedMessage - implements ProtoTestEventOrBuilder { - // Use ProtoTestEvent.newBuilder() to construct. - private ProtoTestEvent(Builder builder) { - super(builder); - } - private ProtoTestEvent(boolean noInit) {} - - private static final ProtoTestEvent defaultInstance; - public static ProtoTestEvent getDefaultInstance() { - return defaultInstance; - } - - public ProtoTestEvent getDefaultInstanceForType() { - return defaultInstance; - } - - @Override - public UnknownFieldSet getUnknownFields() - { - return UnknownFieldSet.getDefaultInstance(); - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; - } - - public enum EventCategory - implements com.google.protobuf.ProtocolMessageEnum { - CATEGORY_ZERO(0, 0), - CATEGORY_ONE(1, 1), - CATEGORY_TWO(2, 2), - ; - - public static final int CATEGORY_ZERO_VALUE = 0; - public static final int CATEGORY_ONE_VALUE = 1; - public static final int CATEGORY_TWO_VALUE = 2; - - - public final int getNumber() { return value; } - - public static EventCategory valueOf(int value) { - switch (value) { - case 0: return CATEGORY_ZERO; - case 1: return CATEGORY_ONE; - case 2: return CATEGORY_TWO; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public EventCategory findValueByNumber(int number) { - return EventCategory.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); - } - - private static final EventCategory[] VALUES = { - CATEGORY_ZERO, CATEGORY_ONE, CATEGORY_TWO, - }; - - public static EventCategory valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - - private final int index; - private final int value; - - private EventCategory(int index, int value) { - this.index = index; - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory) - } - - private int bitField0_; - // required .prototest.ProtoTestEvent.EventCategory eventType = 1; - public static final int EVENTTYPE_FIELD_NUMBER = 1; - private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_; - public boolean hasEventType() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { - return eventType_; - } - - // required uint64 id = 2; - public static final int ID_FIELD_NUMBER = 2; - private long id_; - public boolean hasId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getId() { - return id_; - } - - // required string timestamp = 3; - public static final int TIMESTAMP_FIELD_NUMBER = 3; - private java.lang.Object timestamp_; - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getTimestamp() { - java.lang.Object ref = timestamp_; - if (ref instanceof String) { - return (String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { - timestamp_ = s; - } - return s; - } - } - private com.google.protobuf.ByteString getTimestampBytes() { - java.lang.Object ref = timestamp_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); - timestamp_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - // optional uint32 someOtherId = 4; - public static final int SOMEOTHERID_FIELD_NUMBER = 4; - private int someOtherId_; - public boolean hasSomeOtherId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public int getSomeOtherId() { - return someOtherId_; - } - - // optional bool isValid = 5; - public static final int ISVALID_FIELD_NUMBER = 5; - private boolean isValid_; - public boolean hasIsValid() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getIsValid() { - return isValid_; - } - - // optional string description = 6; - public static final int DESCRIPTION_FIELD_NUMBER = 6; - private java.lang.Object description_; - public boolean hasDescription() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getDescription() { - java.lang.Object ref = description_; - if (ref instanceof String) { - return (String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - String s = bs.toStringUtf8(); - if (com.google.protobuf.Internal.isValidUtf8(bs)) { - description_ = s; - } - return s; - } - } - private com.google.protobuf.ByteString getDescriptionBytes() { - java.lang.Object ref = description_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8((String) ref); - description_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - // optional float someFloatColumn = 7; - public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7; - private float someFloatColumn_; - public boolean hasSomeFloatColumn() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public float getSomeFloatColumn() { - return someFloatColumn_; - } - - // optional uint32 someIntColumn = 8; - public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8; - private int someIntColumn_; - public boolean hasSomeIntColumn() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public int getSomeIntColumn() { - return someIntColumn_; - } - - // optional uint64 someLongColumn = 9; - public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9; - private long someLongColumn_; - public boolean hasSomeLongColumn() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public long getSomeLongColumn() { - return someLongColumn_; - } - - private void initFields() { - eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - id_ = 0L; - timestamp_ = ""; - someOtherId_ = 0; - isValid_ = false; - description_ = ""; - someFloatColumn_ = 0F; - someIntColumn_ = 0; - someLongColumn_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) { - return isInitialized == 1; - } - - if (!hasEventType()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasTimestamp()) { - memoizedIsInitialized = 0; - return false; - } - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeEnum(1, eventType_.getNumber()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, id_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getTimestampBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt32(4, someOtherId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBool(5, isValid_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeBytes(6, getDescriptionBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeFloat(7, someFloatColumn_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeUInt32(8, someIntColumn_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeUInt64(9, someLongColumn_); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) { - return size; - } - - size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, eventType_.getNumber()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, id_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(3, getTimestampBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(4, someOtherId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(5, isValid_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(6, getDescriptionBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += com.google.protobuf.CodedOutputStream - .computeFloatSize(7, someFloatColumn_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(8, someIntColumn_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(9, someLongColumn_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(ProtoTestEventWrapper.ProtoTestEvent prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements ProtoTestEventWrapper.ProtoTestEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; - } - - // Construct using io.druid.data.input.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder(BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - bitField0_ = (bitField0_ & ~0x00000001); - id_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - timestamp_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - someOtherId_ = 0; - bitField0_ = (bitField0_ & ~0x00000008); - isValid_ = false; - bitField0_ = (bitField0_ & ~0x00000010); - description_ = ""; - bitField0_ = (bitField0_ & ~0x00000020); - someFloatColumn_ = 0F; - bitField0_ = (bitField0_ & ~0x00000040); - someIntColumn_ = 0; - bitField0_ = (bitField0_ & ~0x00000080); - someLongColumn_ = 0L; - bitField0_ = (bitField0_ & ~0x00000100); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor(); - } - - public ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() { - return ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); - } - - public ProtoTestEventWrapper.ProtoTestEvent build() { - ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); - if (!result.isInitialized()) { - throw AbstractMessage.Builder.newUninitializedMessageException(result); - } - return result; - } - - private ProtoTestEventWrapper.ProtoTestEvent buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); - if (!result.isInitialized()) { - throw AbstractMessage.Builder.newUninitializedMessageException( - result - ).asInvalidProtocolBufferException(); - } - return result; - } - - public ProtoTestEventWrapper.ProtoTestEvent buildPartial() { - ProtoTestEventWrapper.ProtoTestEvent result = new ProtoTestEventWrapper.ProtoTestEvent(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.eventType_ = eventType_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.id_ = id_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.timestamp_ = timestamp_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.someOtherId_ = someOtherId_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.isValid_ = isValid_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.description_ = description_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.someFloatColumn_ = someFloatColumn_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.someIntColumn_ = someIntColumn_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000100; - } - result.someLongColumn_ = someLongColumn_; - result.bitField0_ = to_bitField0_; - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof ProtoTestEventWrapper.ProtoTestEvent) { - return mergeFrom((ProtoTestEventWrapper.ProtoTestEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(ProtoTestEventWrapper.ProtoTestEvent other) { - if (other == ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) { - return this; - } - if (other.hasEventType()) { - setEventType(other.getEventType()); - } - if (other.hasId()) { - setId(other.getId()); - } - if (other.hasTimestamp()) { - setTimestamp(other.getTimestamp()); - } - if (other.hasSomeOtherId()) { - setSomeOtherId(other.getSomeOtherId()); - } - if (other.hasIsValid()) { - setIsValid(other.getIsValid()); - } - if (other.hasDescription()) { - setDescription(other.getDescription()); - } - if (other.hasSomeFloatColumn()) { - setSomeFloatColumn(other.getSomeFloatColumn()); - } - if (other.hasSomeIntColumn()) { - setSomeIntColumn(other.getSomeIntColumn()); - } - if (other.hasSomeLongColumn()) { - setSomeLongColumn(other.getSomeLongColumn()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - if (!hasEventType()) { - - return false; - } - if (!hasId()) { - - return false; - } - if (!hasTimestamp()) { - - return false; - } - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 8: { - int rawValue = input.readEnum(); - ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(1, rawValue); - } else { - bitField0_ |= 0x00000001; - eventType_ = value; - } - break; - } - case 16: { - bitField0_ |= 0x00000002; - id_ = input.readUInt64(); - break; - } - case 26: { - bitField0_ |= 0x00000004; - timestamp_ = input.readBytes(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - someOtherId_ = input.readUInt32(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - isValid_ = input.readBool(); - break; - } - case 50: { - bitField0_ |= 0x00000020; - description_ = input.readBytes(); - break; - } - case 61: { - bitField0_ |= 0x00000040; - someFloatColumn_ = input.readFloat(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - someIntColumn_ = input.readUInt32(); - break; - } - case 72: { - bitField0_ |= 0x00000100; - someLongColumn_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required .prototest.ProtoTestEvent.EventCategory eventType = 1; - private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - public boolean hasEventType() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { - return eventType_; - } - public Builder setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - eventType_ = value; - onChanged(); - return this; - } - public Builder clearEventType() { - bitField0_ = (bitField0_ & ~0x00000001); - eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - onChanged(); - return this; - } - - // required uint64 id = 2; - private long id_ ; - public boolean hasId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getId() { - return id_; - } - public Builder setId(long value) { - bitField0_ |= 0x00000002; - id_ = value; - onChanged(); - return this; - } - public Builder clearId() { - bitField0_ = (bitField0_ & ~0x00000002); - id_ = 0L; - onChanged(); - return this; - } - - // required string timestamp = 3; - private java.lang.Object timestamp_ = ""; - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getTimestamp() { - java.lang.Object ref = timestamp_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - timestamp_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTimestamp(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - timestamp_ = value; - onChanged(); - return this; - } - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000004); - timestamp_ = getDefaultInstance().getTimestamp(); - onChanged(); - return this; - } - void setTimestamp(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000004; - timestamp_ = value; - onChanged(); - } - - // optional uint32 someOtherId = 4; - private int someOtherId_ ; - public boolean hasSomeOtherId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public int getSomeOtherId() { - return someOtherId_; - } - public Builder setSomeOtherId(int value) { - bitField0_ |= 0x00000008; - someOtherId_ = value; - onChanged(); - return this; - } - public Builder clearSomeOtherId() { - bitField0_ = (bitField0_ & ~0x00000008); - someOtherId_ = 0; - onChanged(); - return this; - } - - // optional bool isValid = 5; - private boolean isValid_ ; - public boolean hasIsValid() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getIsValid() { - return isValid_; - } - public Builder setIsValid(boolean value) { - bitField0_ |= 0x00000010; - isValid_ = value; - onChanged(); - return this; - } - public Builder clearIsValid() { - bitField0_ = (bitField0_ & ~0x00000010); - isValid_ = false; - onChanged(); - return this; - } - - // optional string description = 6; - private java.lang.Object description_ = ""; - public boolean hasDescription() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getDescription() { - java.lang.Object ref = description_; - if (!(ref instanceof String)) { - String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); - description_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setDescription(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; - description_ = value; - onChanged(); - return this; - } - public Builder clearDescription() { - bitField0_ = (bitField0_ & ~0x00000020); - description_ = getDefaultInstance().getDescription(); - onChanged(); - return this; - } - void setDescription(com.google.protobuf.ByteString value) { - bitField0_ |= 0x00000020; - description_ = value; - onChanged(); - } - - // optional float someFloatColumn = 7; - private float someFloatColumn_ ; - public boolean hasSomeFloatColumn() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public float getSomeFloatColumn() { - return someFloatColumn_; - } - public Builder setSomeFloatColumn(float value) { - bitField0_ |= 0x00000040; - someFloatColumn_ = value; - onChanged(); - return this; - } - public Builder clearSomeFloatColumn() { - bitField0_ = (bitField0_ & ~0x00000040); - someFloatColumn_ = 0F; - onChanged(); - return this; - } - - // optional uint32 someIntColumn = 8; - private int someIntColumn_ ; - public boolean hasSomeIntColumn() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public int getSomeIntColumn() { - return someIntColumn_; - } - public Builder setSomeIntColumn(int value) { - bitField0_ |= 0x00000080; - someIntColumn_ = value; - onChanged(); - return this; - } - public Builder clearSomeIntColumn() { - bitField0_ = (bitField0_ & ~0x00000080); - someIntColumn_ = 0; - onChanged(); - return this; - } - - // optional uint64 someLongColumn = 9; - private long someLongColumn_ ; - public boolean hasSomeLongColumn() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public long getSomeLongColumn() { - return someLongColumn_; - } - public Builder setSomeLongColumn(long value) { - bitField0_ |= 0x00000100; - someLongColumn_ = value; - onChanged(); - return this; - } - public Builder clearSomeLongColumn() { - bitField0_ = (bitField0_ & ~0x00000100); - someLongColumn_ = 0L; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent) - } - - static { - defaultInstance = new ProtoTestEvent(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent) - } - - private static com.google.protobuf.Descriptors.Descriptor - internal_static_prototest_ProtoTestEvent_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_prototest_ProtoTestEvent_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n\017ProtoTest.proto\022\tprototest\"\266\002\n\016ProtoTe" + - "stEvent\022:\n\teventType\030\001 \002(\0162\'.prototest.P" + - "rotoTestEvent.EventCategory\022\n\n\002id\030\002 \002(\004\022" + - "\021\n\ttimestamp\030\003 \002(\t\022\023\n\013someOtherId\030\004 \001(\r\022" + - "\017\n\007isValid\030\005 \001(\010\022\023\n\013description\030\006 \001(\t\022\027\n" + - "\017someFloatColumn\030\007 \001(\002\022\025\n\rsomeIntColumn\030" + - "\010 \001(\r\022\026\n\016someLongColumn\030\t \001(\004\"F\n\rEventCa" + - "tegory\022\021\n\rCATEGORY_ZERO\020\000\022\020\n\014CATEGORY_ON" + - "E\020\001\022\020\n\014CATEGORY_TWO\020\002B6\n\035com.metamx.drui" + - "d.indexer.dataB\025ProtoTestEventWrapper" - }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { - descriptor = root; - internal_static_prototest_ProtoTestEvent_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_prototest_ProtoTestEvent_descriptor, - new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", }, - ProtoTestEventWrapper.ProtoTestEvent.class, - ProtoTestEventWrapper.ProtoTestEvent.Builder.class); - return null; - } - }; - com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }, assigner); - } - - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java index 44abae08f0bd..cbeeff9d323c 100644 --- a/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java +++ b/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.granularity.DurationGranularity; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.java.util.common.granularity.GranularityType; import io.druid.java.util.common.granularity.PeriodGranularity; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -104,7 +105,7 @@ public void testIterable15MinuteSimple() throws Exception new DateTime("2011-01-01T09:45:00.000Z"), new DateTime("2011-01-01T10:00:00.000Z") ), - Granularities.FIFTEEN_MINUTE.getIterable( + Granularities.FIFTEEN_MINUTE.getIterable( new Interval( baseTime.getMillis(), baseTime.plus(Minutes.minutes(45)).getMillis() )) @@ -123,7 +124,10 @@ public void testIterable15MinuteComplex() throws Exception new DateTime("2011-01-01T10:00:00.000Z"), new DateTime("2011-01-01T10:15:00.000Z") ), - Granularities.FIFTEEN_MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.minutes(45)).getMillis())) + Granularities.FIFTEEN_MINUTE.getIterable(new Interval( + baseTime.getMillis(), + baseTime.plus(Minutes.minutes(45)).getMillis() + )) ); } @@ -318,9 +322,9 @@ public void testPeriodDaylightSaving() throws Exception final DateTime baseTime = new DateTime("2012-11-04T00:00:00", tz); assertSameInterval( Lists.newArrayList( - new DateTime("2012-11-04T00:00:00.000-07:00"), - new DateTime("2012-11-05T00:00:00.000-08:00"), - new DateTime("2012-11-06T00:00:00.000-08:00") + new DateTime("2012-11-04T00:00:00.000-07:00", tz), + new DateTime("2012-11-05T00:00:00.000-08:00", tz), + new DateTime("2012-11-06T00:00:00.000-08:00", tz) ), new PeriodGranularity(new Period("P1D"), null, tz) .getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis())) @@ -328,11 +332,11 @@ public void testPeriodDaylightSaving() throws Exception assertSameInterval( Lists.newArrayList( - new DateTime("2012-11-04T00:00:00.000-07:00"), - new DateTime("2012-11-04T01:00:00.000-07:00"), - new DateTime("2012-11-04T01:00:00.000-08:00"), - new DateTime("2012-11-04T02:00:00.000-08:00"), - new DateTime("2012-11-04T03:00:00.000-08:00") + new DateTime("2012-11-04T00:00:00.000-07:00", tz), + new DateTime("2012-11-04T01:00:00.000-07:00", tz), + new DateTime("2012-11-04T01:00:00.000-08:00", tz), + new DateTime("2012-11-04T02:00:00.000-08:00", tz), + new DateTime("2012-11-04T03:00:00.000-08:00", tz) ), new PeriodGranularity(new Period("PT1H"), null, tz) .getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Hours.hours(5)).getMillis())) @@ -341,18 +345,18 @@ public void testPeriodDaylightSaving() throws Exception final PeriodGranularity hour = new PeriodGranularity(new Period("PT1H"), null, tz); assertSameDateTime( Lists.newArrayList( - new DateTime("2012-11-04T00:00:00.000-07:00"), - new DateTime("2012-11-04T01:00:00.000-07:00"), - new DateTime("2012-11-04T01:00:00.000-08:00"), - new DateTime("2012-11-04T02:00:00.000-08:00"), - new DateTime("2012-11-04T03:00:00.000-08:00") + new DateTime("2012-11-04T00:00:00.000-07:00", tz), + new DateTime("2012-11-04T01:00:00.000-07:00", tz), + new DateTime("2012-11-04T01:00:00.000-08:00", tz), + new DateTime("2012-11-04T02:00:00.000-08:00", tz), + new DateTime("2012-11-04T03:00:00.000-08:00", tz) ), Lists.newArrayList( - hour.bucketStart(new DateTime("2012-11-04T00:30:00-07:00")), - hour.bucketStart(new DateTime("2012-11-04T01:30:00-07:00")), - hour.bucketStart(new DateTime("2012-11-04T01:30:00-08:00")), - hour.bucketStart(new DateTime("2012-11-04T02:30:00-08:00")), - hour.bucketStart(new DateTime("2012-11-04T03:30:00-08:00")) + hour.bucketStart(new DateTime("2012-11-04T00:30:00-07:00")), + hour.bucketStart(new DateTime("2012-11-04T01:30:00-07:00")), + hour.bucketStart(new DateTime("2012-11-04T01:30:00-08:00")), + hour.bucketStart(new DateTime("2012-11-04T02:30:00-08:00")), + hour.bucketStart(new DateTime("2012-11-04T03:30:00-08:00")) ) ); } @@ -364,10 +368,10 @@ public void testIterableMonth() throws Exception final DateTime baseTime = new DateTime("2012-11-03T10:00:00", tz); assertSameInterval( Lists.newArrayList( - new DateTime("2012-11-01T00:00:00.000-07:00"), - new DateTime("2012-12-01T00:00:00.000-08:00"), - new DateTime("2013-01-01T00:00:00.000-08:00"), - new DateTime("2013-02-01T00:00:00.000-08:00") + new DateTime("2012-11-01T00:00:00.000-07:00", tz), + new DateTime("2012-12-01T00:00:00.000-08:00", tz), + new DateTime("2013-01-01T00:00:00.000-08:00", tz), + new DateTime("2013-02-01T00:00:00.000-08:00", tz) ), new PeriodGranularity(new Period("P1M"), null, tz) .getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.months(3)).getMillis())) @@ -381,10 +385,10 @@ public void testIterableWeek() throws Exception final DateTime baseTime = new DateTime("2012-11-03T10:00:00", tz); assertSameInterval( Lists.newArrayList( - new DateTime("2012-10-29T00:00:00.000-07:00"), - new DateTime("2012-11-05T00:00:00.000-08:00"), - new DateTime("2012-11-12T00:00:00.000-08:00"), - new DateTime("2012-11-19T00:00:00.000-08:00") + new DateTime("2012-10-29T00:00:00.000-07:00", tz), + new DateTime("2012-11-05T00:00:00.000-08:00", tz), + new DateTime("2012-11-12T00:00:00.000-08:00", tz), + new DateTime("2012-11-19T00:00:00.000-08:00", tz) ), new PeriodGranularity(new Period("P1W"), null, tz) .getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Weeks.weeks(3)).getMillis())) @@ -392,9 +396,9 @@ public void testIterableWeek() throws Exception assertSameInterval( Lists.newArrayList( - new DateTime("2012-11-03T10:00:00.000-07:00"), - new DateTime("2012-11-10T10:00:00.000-08:00"), - new DateTime("2012-11-17T10:00:00.000-08:00") + new DateTime("2012-11-03T10:00:00.000-07:00", tz), + new DateTime("2012-11-10T10:00:00.000-08:00", tz), + new DateTime("2012-11-17T10:00:00.000-08:00", tz) ), new PeriodGranularity(new Period("P1W"), baseTime, tz) .getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Weeks.weeks(3)).getMillis())) @@ -404,15 +408,18 @@ public void testIterableWeek() throws Exception @Test public void testPeriodTruncateDays() throws Exception { + final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); final DateTime origin = new DateTime("2012-01-02T05:00:00.000-08:00"); - PeriodGranularity periodOrigin = new PeriodGranularity(new Period("P2D"), - origin, - DateTimeZone.forID("America/Los_Angeles")); + PeriodGranularity periodOrigin = new PeriodGranularity( + new Period("P2D"), + origin, + tz + ); assertSameDateTime( Lists.newArrayList( - new DateTime("2011-12-31T05:00:00.000-08:00"), - new DateTime("2012-01-02T05:00:00.000-08:00"), - new DateTime("2012-01-04T05:00:00.000-08:00") + new DateTime("2011-12-31T05:00:00.000-08:00", tz), + new DateTime("2012-01-02T05:00:00.000-08:00", tz), + new DateTime("2012-01-04T05:00:00.000-08:00", tz) ), Lists.newArrayList( periodOrigin.bucketStart(new DateTime("2012-01-01T05:00:04.123-08:00")), @@ -422,14 +429,16 @@ public void testPeriodTruncateDays() throws Exception ) ); - PeriodGranularity periodNoOrigin = new PeriodGranularity(new Period("P2D"), - null, - DateTimeZone.forID("America/Los_Angeles")); + PeriodGranularity periodNoOrigin = new PeriodGranularity( + new Period("P2D"), + null, + tz + ); assertSameDateTime( Lists.newArrayList( - new DateTime("2012-01-01T00:00:00.000-08:00"), - new DateTime("2012-01-01T00:00:00.000-08:00"), - new DateTime("2012-01-03T00:00:00.000-08:00") + new DateTime("2012-01-01T00:00:00.000-08:00", tz), + new DateTime("2012-01-01T00:00:00.000-08:00", tz), + new DateTime("2012-01-03T00:00:00.000-08:00", tz) ), Lists.newArrayList( periodNoOrigin.bucketStart(new DateTime("2012-01-01T05:00:04.123-08:00")), @@ -479,18 +488,19 @@ public void testPeriodTruncateMinutes() throws Exception public void testCompoundPeriodTruncate() throws Exception { { + final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); final DateTime origin = new DateTime("2012-01-02T05:00:00.000-08:00"); PeriodGranularity periodOrigin = new PeriodGranularity( new Period("P1M2D"), origin, - DateTimeZone.forID("America/Los_Angeles") + tz ); assertSameDateTime( Lists.newArrayList( - new DateTime("2011-11-30T05:00:00.000-08:00"), - new DateTime("2012-01-02T05:00:00.000-08:00"), - new DateTime("2012-02-04T05:00:00.000-08:00"), - new DateTime("2012-02-04T05:00:00.000-08:00") + new DateTime("2011-11-30T05:00:00.000-08:00", tz), + new DateTime("2012-01-02T05:00:00.000-08:00", tz), + new DateTime("2012-02-04T05:00:00.000-08:00", tz), + new DateTime("2012-02-04T05:00:00.000-08:00", tz) ), Lists.newArrayList( periodOrigin.bucketStart(new DateTime("2012-01-01T05:00:04.123-08:00")), @@ -503,14 +513,14 @@ public void testCompoundPeriodTruncate() throws Exception PeriodGranularity periodNoOrigin = new PeriodGranularity( new Period("P1M2D"), null, - DateTimeZone.forID("America/Los_Angeles") + tz ); assertSameDateTime( Lists.newArrayList( - new DateTime("1970-01-01T00:00:00.000-08:00"), - new DateTime("2011-12-12T00:00:00.000-08:00"), - new DateTime("2012-01-14T00:00:00.000-08:00"), - new DateTime("2012-02-16T00:00:00.000-08:00") + new DateTime("1970-01-01T00:00:00.000-08:00", tz), + new DateTime("2011-12-12T00:00:00.000-08:00", tz), + new DateTime("2012-01-14T00:00:00.000-08:00", tz), + new DateTime("2012-02-16T00:00:00.000-08:00", tz) ), Lists.newArrayList( periodNoOrigin.bucketStart(new DateTime("1970-01-01T05:02:04.123-08:00")), @@ -523,18 +533,19 @@ public void testCompoundPeriodTruncate() throws Exception } { + final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles"); final DateTime origin = new DateTime("2012-01-02T05:00:00.000-08:00"); PeriodGranularity periodOrigin = new PeriodGranularity( new Period("PT12H5M"), origin, - DateTimeZone.forID("America/Los_Angeles") + tz ); assertSameDateTime( Lists.newArrayList( - new DateTime("2012-01-01T04:50:00.000-08:00"), - new DateTime("2012-01-02T05:00:00.000-08:00"), - new DateTime("2012-01-02T17:05:00.000-08:00"), - new DateTime("2012-02-03T22:25:00.000-08:00") + new DateTime("2012-01-01T04:50:00.000-08:00", tz), + new DateTime("2012-01-02T05:00:00.000-08:00", tz), + new DateTime("2012-01-02T17:05:00.000-08:00", tz), + new DateTime("2012-02-03T22:25:00.000-08:00", tz) ), Lists.newArrayList( periodOrigin.bucketStart(new DateTime("2012-01-01T05:00:04.123-08:00")), @@ -640,16 +651,26 @@ public void testIterableAllComplex() throws Exception @Test public void testSerializePeriod() throws Exception { - ObjectMapper mapper = new DefaultObjectMapper(); + final ObjectMapper mapper = new DefaultObjectMapper(); String json = "{ \"type\": \"period\", \"period\": \"P1D\" }"; Granularity gran = mapper.readValue(json, Granularity.class); Assert.assertEquals(new PeriodGranularity(new Period("P1D"), null, null), gran); - json = "{ \"type\": \"period\", \"period\": \"P1D\"," + // Nonstandard period + json = "{ \"type\": \"period\", \"period\": \"P2D\" }"; + gran = mapper.readValue(json, Granularity.class); + Assert.assertEquals(new PeriodGranularity(new Period("P2D"), null, null), gran); + + // Set timeZone, origin + json = "{ \"type\": \"period\", \"period\": \"P1D\"," + "\"timeZone\": \"America/Los_Angeles\", \"origin\": \"1970-01-01T00:00:00Z\"}"; gran = mapper.readValue(json, Granularity.class); - Assert.assertEquals(new PeriodGranularity(new Period("P1D"), new DateTime(0L), DateTimeZone.forID("America/Los_Angeles")), gran); + Assert.assertEquals(new PeriodGranularity( + new Period("P1D"), + new DateTime(0L), + DateTimeZone.forID("America/Los_Angeles") + ), gran); PeriodGranularity expected = new PeriodGranularity( new Period("P1D"), @@ -695,46 +716,40 @@ public void testSerializeDuration() throws Exception } @Test - public void testSerializeSimple() throws Exception + public void testStandardGranularitiesSerde() throws Exception { ObjectMapper mapper = new DefaultObjectMapper(); - Assert.assertEquals( - Granularities.ALL, - mapper.readValue( - mapper.writeValueAsString(Granularities.ALL), - Granularity.class - ) - ); + for (GranularityType granularityType : GranularityType.values()) { + final Granularity granularity = granularityType.getDefaultGranularity(); - Assert.assertEquals( - Granularities.NONE, - mapper.readValue( - mapper.writeValueAsString(Granularities.NONE), - Granularity.class - ) - ); - } - - @Test - public void testDeserializeSimple() throws Exception - { - ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals( + granularity, + mapper.readValue("\"" + granularityType.name().toUpperCase() + "\"", Granularity.class) + ); - Assert.assertEquals(Granularities.ALL, mapper.readValue("\"all\"", Granularity.class)); - Assert.assertEquals(Granularities.ALL, mapper.readValue("\"ALL\"", Granularity.class)); - Assert.assertEquals(Granularities.NONE, mapper.readValue("\"none\"", Granularity.class)); - Assert.assertEquals(Granularities.NONE, mapper.readValue("\"NONE\"", Granularity.class)); + Assert.assertEquals( + granularity, + mapper.readValue("\"" + granularityType.name().toLowerCase() + "\"", Granularity.class) + ); - Assert.assertEquals(Granularities.DAY, mapper.readValue("\"day\"", Granularity.class)); - Assert.assertEquals(Granularities.HOUR, mapper.readValue("\"hour\"", Granularity.class)); - Assert.assertEquals(Granularities.MINUTE, mapper.readValue("\"minute\"", Granularity.class)); - Assert.assertEquals(Granularities.FIFTEEN_MINUTE, mapper.readValue("\"fifteen_minute\"", Granularity.class)); + Assert.assertEquals( + granularity, + mapper.readValue(mapper.writeValueAsString(granularity), Granularity.class) + ); - Assert.assertEquals(Granularities.WEEK, mapper.readValue("\"week\"", Granularity.class)); - Assert.assertEquals(Granularities.QUARTER, mapper.readValue("\"quarter\"", Granularity.class)); - Assert.assertEquals(Granularities.MONTH, mapper.readValue("\"month\"", Granularity.class)); - Assert.assertEquals(Granularities.YEAR, mapper.readValue("\"year\"", Granularity.class)); + if (granularityType == GranularityType.ALL || granularityType == GranularityType.NONE) { + Assert.assertEquals( + "{\"type\":\"" + granularityType.name().toLowerCase() + "\"}", + mapper.writeValueAsString(granularity) + ); + } else { + Assert.assertEquals( + "\"" + granularityType.name().toUpperCase() + "\"", + mapper.writeValueAsString(granularity) + ); + } + } } @Test @@ -745,7 +760,7 @@ public void testMerge() Assert.assertNull(Granularity.mergeGranularities(Lists.newArrayList(null, Granularities.DAY))); Assert.assertNull(Granularity.mergeGranularities(Lists.newArrayList(Granularities.DAY, null))); Assert.assertNull( - Granularity.mergeGranularities( + Granularity.mergeGranularities( Lists.newArrayList( Granularities.DAY, null, @@ -754,7 +769,7 @@ public void testMerge() ) ); Assert.assertNull( - Granularity.mergeGranularities(ImmutableList.of(Granularities.ALL, Granularities.DAY)) + Granularity.mergeGranularities(ImmutableList.of(Granularities.ALL, Granularities.DAY)) ); Assert.assertEquals( @@ -792,9 +807,9 @@ private void assertSameInterval(List expected, Iterable actu @Test(timeout = 10_000L) public void testDeadLock() throws Exception { - final URL[] urls = ((URLClassLoader)Granularity.class.getClassLoader()).getURLs(); + final URL[] urls = ((URLClassLoader) Granularity.class.getClassLoader()).getURLs(); final String className = Granularity.class.getCanonicalName(); - for(int i = 0; i < 1000; ++i) { + for (int i = 0; i < 1000; ++i) { final ClassLoader loader = new URLClassLoader(urls, null); Assert.assertNotNull(String.valueOf(i), Class.forName(className, true, loader)); } diff --git a/processing/src/test/java/io/druid/guice/GuiceInjectorsTest.java b/processing/src/test/java/io/druid/guice/GuiceInjectorsTest.java index 145459f89901..10352d14293c 100644 --- a/processing/src/test/java/io/druid/guice/GuiceInjectorsTest.java +++ b/processing/src/test/java/io/druid/guice/GuiceInjectorsTest.java @@ -63,7 +63,7 @@ public String getValue() } } - private class CustomEmitterFactory implements Provider { + private static class CustomEmitterFactory implements Provider { private Emitter emitter; private Injector injector; diff --git a/processing/src/test/java/io/druid/guice/SegmentMetadataQueryConfigTest.java b/processing/src/test/java/io/druid/guice/SegmentMetadataQueryConfigTest.java new file mode 100644 index 000000000000..68212d8f7081 --- /dev/null +++ b/processing/src/test/java/io/druid/guice/SegmentMetadataQueryConfigTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.Provides; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.metadata.SegmentMetadataQueryConfig; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.EnumSet; +import java.util.Iterator; +import java.util.Properties; + +public class SegmentMetadataQueryConfigTest +{ + @Test + public void testSerdeSegmentMetadataQueryConfig() throws Exception + { + Injector injector = Guice.createInjector( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.install(new PropertiesModule(Arrays.asList("test.runtime.properties"))); + binder.install(new ConfigModule()); + binder.install(new DruidGuiceExtensions()); + JsonConfigProvider.bind(binder, "druid.query.segmentMetadata", SegmentMetadataQueryConfig.class); + } + + @Provides + @LazySingleton + public ObjectMapper jsonMapper() + { + return new DefaultObjectMapper(); + } + } + ); + + + Properties props = injector.getInstance(Properties.class); + SegmentMetadataQueryConfig config = injector.getInstance(SegmentMetadataQueryConfig.class); + + EnumSet expectedDefaultAnalysis = config.getDefaultAnalysisTypes(); + String actualDefaultAnalysis = props.getProperty("druid.query.segmentMetadata.defaultAnalysisTypes"); + + Iterator it = expectedDefaultAnalysis.iterator(); + StringBuilder sb = new StringBuilder(); + sb.append('['); + while (it.hasNext()) { + SegmentMetadataQuery.AnalysisType e = it.next(); + sb.append("\""+e+"\""); + if (it.hasNext()) { + sb.append(',').append(' '); + } + } + sb.append(']'); + + String expectedDefaultAnalysisAsString = sb.toString(); + + Assert.assertEquals( + expectedDefaultAnalysisAsString, + actualDefaultAnalysis + ); + Assert.assertEquals( + props.getProperty("druid.query.segmentMetadata.defaultHistory"), + config.getDefaultHistory().toString() + ); + } +} diff --git a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java index b6c074e7eeb6..a5761454f635 100644 --- a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java @@ -61,7 +61,7 @@ public void testAsyncNature() throws Exception { QueryRunner baseRunner = new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { try { latch.await(); @@ -85,7 +85,7 @@ public void testQueryTimeoutHonored() { QueryRunner baseRunner = new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { try { Thread.sleep(Long.MAX_VALUE); @@ -117,7 +117,7 @@ public void testQueryRegistration() { QueryRunner baseRunner = new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) { return null; } + public Sequence run(QueryPlus queryPlus, Map responseContext) { return null; } }; QueryWatcher mock = EasyMock.createMock(QueryWatcher.class); diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 311aaaa798f9..935957eb079d 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -330,7 +330,7 @@ public DyingQueryRunner(CountDownLatch start, CountDownLatch stop, Queue run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { // do a lot of work synchronized (this) { diff --git a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java index 194b12e6bdc5..6ec306db2fc8 100644 --- a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java @@ -60,13 +60,13 @@ public void setup() { @Test public void testDefaultNoChunking() { - Query query = queryBuilder.intervals("2014/2016").build(); + QueryPlus queryPlus = QueryPlus.wrap(queryBuilder.intervals("2014/2016").build()); - EasyMock.expect(baseRunner.run(query, Collections.EMPTY_MAP)).andReturn(Sequences.empty()); + EasyMock.expect(baseRunner.run(queryPlus, Collections.EMPTY_MAP)).andReturn(Sequences.empty()); EasyMock.replay(baseRunner); QueryRunner runner = decorator.decorate(baseRunner, toolChest); - runner.run(query, Collections.EMPTY_MAP); + runner.run(queryPlus, Collections.EMPTY_MAP); EasyMock.verify(baseRunner); } diff --git a/processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java b/processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java new file mode 100644 index 000000000000..31bf7825fc4c --- /dev/null +++ b/processing/src/test/java/io/druid/query/LegacyApiQueryRunnerTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query; + +import com.google.common.collect.ImmutableList; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +/** + * Tests that if a QueryRunner overrides a legacy {@link QueryRunner#run(Query, Map)} method, it still works. This + * test should be removed when {@link QueryRunner#run(Query, Map)} is removed. + */ +public class LegacyApiQueryRunnerTest +{ + private static class LegacyApiQueryRunner implements QueryRunner + { + /** + * Overrides legacy API. + */ + @Override + public Sequence run(Query query, Map responseContext) + { + return Sequences.empty(); + } + } + + @Test + public void testQueryRunnerLegacyApi() + { + final Query query = new TestQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(ImmutableList.of(new Interval("0/100"))), + false, + new HashMap() + ); + + Map context = new HashMap<>(); + Assert.assertEquals(Sequences.empty(), new LegacyApiQueryRunner<>().run(QueryPlus.wrap(query), context)); + } +} diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index d4fb1223ec82..614d45ff18b0 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -130,7 +130,9 @@ public static void setupClass() throws Exception new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("product", "tags")), null, null), "\t", - ImmutableList.of("timestamp", "product", "tags") + ImmutableList.of("timestamp", "product", "tags"), + false, + 0 ), "UTF-8" ); diff --git a/processing/src/test/java/io/druid/query/QueryContextsTest.java b/processing/src/test/java/io/druid/query/QueryContextsTest.java index 2a6945b9d3cf..81f23374a21e 100644 --- a/processing/src/test/java/io/druid/query/QueryContextsTest.java +++ b/processing/src/test/java/io/druid/query/QueryContextsTest.java @@ -21,67 +21,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.HashMap; -import java.util.Map; public class QueryContextsTest { - private static class TestQuery extends BaseQuery - { - - public TestQuery(DataSource dataSource, QuerySegmentSpec querySegmentSpec, boolean descending, Map context) - { - super(dataSource, querySegmentSpec, descending, context); - } - - @Override - public boolean hasFilters() - { - return false; - } - - @Override - public DimFilter getFilter() - { - return null; - } - - @Override - public String getType() - { - return null; - } - - @Override - public Query withQuerySegmentSpec(QuerySegmentSpec spec) - { - return null; - } - - @Override - public Query withDataSource(DataSource dataSource) - { - return null; - } - - @Override - public Query withOverriddenContext(Map contextOverride) - { - return new TestQuery( - getDataSource(), - getQuerySegmentSpec(), - isDescending(), - BaseQuery.computeOverriddenContext(getContext(), contextOverride) - ); - } - } @Test public void testDefaultQueryTimeout() diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index c1206adae34d..a24d92b5a4e2 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -501,9 +501,9 @@ public static QueryRunner makeUnionQueryRunner( return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - return qr.run(query, responseContext); + return qr.run(queryPlus, responseContext); } @Override @@ -526,8 +526,9 @@ public static QueryRunner makeFilteringQueryRunner( new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { + Query query = queryPlus.getQuery(); List segments = Lists.newArrayList(); for (Interval interval : query.getIntervals()) { segments.addAll(timeline.lookup(interval)); @@ -535,7 +536,7 @@ public Sequence run(Query query, Map responseContext) List> sequences = Lists.newArrayList(); for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { Segment segment = holder.getObject().getChunk(0).getObject(); - Query running = query.withQuerySegmentSpec( + QueryPlus queryPlusRunning = queryPlus.withQuerySegmentSpec( new SpecificSegmentSpec( new SegmentDescriptor( holder.getInterval(), @@ -544,7 +545,7 @@ public Sequence run(Query query, Map responseContext) ) ) ); - sequences.add(factory.createRunner(segment).run(running, responseContext)); + sequences.add(factory.createRunner(segment).run(queryPlusRunning, responseContext)); } return new MergeSequence<>(query.getResultOrdering(), Sequences.simple(sequences)); } @@ -568,9 +569,9 @@ public QueryRunner decorate( return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - return delegate.run(query, responseContext); + return delegate.run(queryPlus, responseContext); } }; } diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index d7ae4b8b7c1c..eb68d47a4eda 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -71,7 +71,7 @@ public void testRunWithMissingSegments() throws Exception new QueryRunner>() { @Override - public Sequence> run(Query query, Map context) + public Sequence> run(QueryPlus queryPlus, Map context) { ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( new SegmentDescriptor( @@ -128,7 +128,7 @@ public void testRetry() throws Exception { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map context ) { @@ -165,6 +165,7 @@ public Sequence> run( private int numTries = 1; private boolean returnPartialResults = true; + @Override public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } @@ -195,7 +196,7 @@ public void testRetryMultiple() throws Exception { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map context ) { @@ -232,6 +233,7 @@ public Sequence> run( private int numTries = 4; private boolean returnPartialResults = true; + @Override public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } @@ -261,7 +263,7 @@ public void testException() throws Exception { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map context ) { @@ -284,6 +286,7 @@ public Sequence> run( private int numTries = 1; private boolean returnPartialResults = false; + @Override public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } @@ -313,10 +316,11 @@ public void testNoDuplicateRetry() throws Exception { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map context ) { + final Query> query = queryPlus.getQuery(); if ((int) context.get("count") == 0) { // assume 2 missing segments at first run ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( @@ -395,6 +399,7 @@ public Sequence> run( private int numTries = 2; private boolean returnPartialResults = false; + @Override public int getNumTries() { return numTries; } public boolean returnPartialResults() { return returnPartialResults; } diff --git a/processing/src/test/java/io/druid/query/TestQuery.java b/processing/src/test/java/io/druid/query/TestQuery.java new file mode 100644 index 000000000000..bd478cc622c5 --- /dev/null +++ b/processing/src/test/java/io/druid/query/TestQuery.java @@ -0,0 +1,75 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query; + +import io.druid.query.filter.DimFilter; +import io.druid.query.spec.QuerySegmentSpec; + +import java.util.Map; + +class TestQuery extends BaseQuery +{ + + public TestQuery(DataSource dataSource, QuerySegmentSpec querySegmentSpec, boolean descending, Map context) + { + super(dataSource, querySegmentSpec, descending, context); + } + + @Override + public boolean hasFilters() + { + return false; + } + + @Override + public DimFilter getFilter() + { + return null; + } + + @Override + public String getType() + { + return null; + } + + @Override + public Query withQuerySegmentSpec(QuerySegmentSpec spec) + { + return null; + } + + @Override + public Query withDataSource(DataSource dataSource) + { + return null; + } + + @Override + public Query withOverriddenContext(Map contextOverride) + { + return new TestQuery( + getDataSource(), + getQuerySegmentSpec(), + isDescending(), + BaseQuery.computeOverriddenContext(getContext(), contextOverride) + ); + } +} diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index 2f8ae2181088..133e435ed97f 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -81,7 +81,7 @@ public void testPostProcess() throws Exception { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map responseContext ) { @@ -96,7 +96,7 @@ public Sequence> run( new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ), new Result<>( - query.getIntervals().get(0).getEnd(), + queryPlus.getQuery().getIntervals().get(0).getEnd(), new TimeseriesResultValue(ImmutableMap.of("metric", 5)) ) ) @@ -143,7 +143,7 @@ public Sequence> run( { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map responseContext ) { @@ -193,10 +193,11 @@ public void testEmptyFutureInterval() throws Exception { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map responseContext ) { + final Query> query = queryPlus.getQuery(); return Sequences.simple( ImmutableList.of( new Result<>( diff --git a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java index b7b914fe5ab4..250006458ba1 100644 --- a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java @@ -39,11 +39,11 @@ public void testUnionQueryRunner() QueryRunner baseRunner = new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { // verify that table datasource is passed to baseQueryRunner - Assert.assertTrue(query.getDataSource() instanceof TableDataSource); - String dsName = Iterables.getOnlyElement(query.getDataSource().getNames()); + Assert.assertTrue(queryPlus.getQuery().getDataSource() instanceof TableDataSource); + String dsName = Iterables.getOnlyElement(queryPlus.getQuery().getDataSource().getNames()); if (dsName.equals("ds1")) { responseContext.put("ds1", "ds1"); return Sequences.simple(Arrays.asList(1, 2, 3)); @@ -70,7 +70,7 @@ public Sequence run(Query query, Map responseContext) .aggregators(QueryRunnerTestHelper.commonAggregators) .build(); Map responseContext = Maps.newHashMap(); - Sequence result = runner.run(q, responseContext); + Sequence result = runner.run(q, responseContext); List res = Sequences.toList(result, Lists.newArrayList()); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), res); diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index df73615e9911..a782e39c5b72 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -50,6 +50,7 @@ import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -535,10 +536,10 @@ public QueryRunner makeStringSerdeQueryRunner(final ObjectMapper mapper, fi return new QueryRunner() { @Override - public Sequence run(Query query, Map map) + public Sequence run(QueryPlus queryPlus, Map map) { try { - Sequence resultSeq = baseRunner.run(query, Maps.newHashMap()); + Sequence resultSeq = baseRunner.run(queryPlus, Maps.newHashMap()); final Yielder yielder = resultSeq.toYielder( null, new YieldingAccumulator() @@ -559,7 +560,7 @@ public Object accumulate(Object accumulated, Object in) List resultRows = Lists.transform( readQueryResultArrayFromString(resultStr), toolChest.makePreComputeManipulatorFn( - query, + queryPlus.getQuery(), MetricManipulatorFns.deserializing() ) ); diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index b2db0636c671..7671700040c8 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -172,6 +172,7 @@ public int lookupId(String name) @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } } ); diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java index 8c022b5a28d4..e8d7992d147c 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java @@ -104,6 +104,7 @@ public float get() return floats[(int) (index % floats.length)]; } + @Override public void increment() { ++index; diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java index f02675d12fca..c26349711b4d 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java @@ -357,7 +357,7 @@ public static void main(String... args) throws Exception ++i; } long t1 = System.currentTimeMillis() - t; - System.out.println(String.format("JavaScript aggregator == %,f: %d ms", aggRhino.get(), t1)); + System.out.println(String.format("JavaScript aggregator == %,f: %d ms", aggRhino.getFloat(), t1)); t = System.currentTimeMillis(); i = 0; @@ -366,7 +366,7 @@ public static void main(String... args) throws Exception ++i; } long t2 = System.currentTimeMillis() - t; - System.out.println(String.format("DoubleSum aggregator == %,f: %d ms", doubleAgg.get(), t2)); + System.out.println(String.format("DoubleSum aggregator == %,f: %d ms", doubleAgg.getFloat(), t2)); System.out.println(String.format("JavaScript is %2.1fx slower", (double) t1 / t2)); } diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java index 718b358ea4be..fcd50885419f 100644 --- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java +++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorBenchmark.java @@ -53,6 +53,7 @@ public class CardinalityAggregatorBenchmark extends SimpleBenchmark @Param({"true", "false"}) boolean byRow; + @Override protected void setUp() { Iterable values = FluentIterable diff --git a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java index a19a7396fcd9..0f53090c482c 100644 --- a/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java @@ -157,7 +157,12 @@ public void fill(int index, int[] toFill) @Override public void close() throws IOException { + } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // Don't care about runtime shape in tests } }; } @@ -210,6 +215,7 @@ public int lookupId(String s) @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } } diff --git a/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java b/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java index e31b51a7ac13..a105b6e2378d 100644 --- a/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java +++ b/processing/src/test/java/io/druid/query/dimension/TestDimensionSelector.java @@ -97,5 +97,6 @@ public int lookupId(String name) @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java index 53e81977e298..82180d295007 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -190,6 +190,7 @@ public ByteBuffer get () GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { + @Override public String getDefaultStrategy() { return "v2"; diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index dc7648154da2..edda6b9db3cd 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -33,6 +33,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.aggregation.AggregatorFactory; @@ -86,21 +87,22 @@ public void testMergeRunnersEnsureGroupMerging() throws Exception new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return factory.getToolchest().mergeResults( new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { + final Query query = queryPlus.getQuery(); try { return new MergeSequence( query.getResultOrdering(), Sequences.simple( Arrays.asList( - factory.createRunner(createSegment()).run(query, responseContext), - factory.createRunner(createSegment()).run(query, responseContext) + factory.createRunner(createSegment()).run(queryPlus, responseContext), + factory.createRunner(createSegment()).run(queryPlus, responseContext) ) ) ); @@ -110,7 +112,7 @@ public Sequence run(Query query, Map responseContext) } } } - ).run(query, responseContext); + ).run(queryPlus, responseContext); } } ); @@ -144,7 +146,9 @@ private Segment createSegment() throws Exception new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("product", "tags")), null, null), "\t", - ImmutableList.of("timestamp", "product", "tags") + ImmutableList.of("timestamp", "product", "tags"), + false, + 0 ), "UTF-8" ); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java index 060012bf0eb5..11dd20b1507a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -154,6 +154,7 @@ public ByteBuffer get () GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { + @Override public String getDefaultStrategy() { return "v2"; @@ -247,7 +248,6 @@ public void testResourceLimitExceededOnBroker() @Test(timeout = 10000, expected = InsufficientResourcesException.class) public void testInsufficientResourcesOnBroker() throws IOException { - final ReferenceCountingResourceHolder> holder = mergeBufferPool.takeBatch(1, 10); final GroupByQuery query = GroupByQuery .builder() .setDataSource( @@ -267,10 +267,8 @@ public void testInsufficientResourcesOnBroker() throws IOException .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500)) .build(); - try { + try (ReferenceCountingResourceHolder> holder = mergeBufferPool.takeBatch(1, 10)) { GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - } finally { - holder.close(); } } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 147c8e1bf64f..ffdcc648d459 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -54,6 +54,7 @@ import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; @@ -2353,20 +2354,20 @@ public void testMergeResults() { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) ) ); } @@ -2649,20 +2650,20 @@ private void doTestMergeResultsWithOrderBy(LimitSpec orderBySpec, List expe { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) ) ); } @@ -3437,20 +3438,20 @@ public void testPostAggMergedHavingSpec() { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) ) ); } @@ -3770,20 +3771,20 @@ public void testMergedHavingSpec() { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) ) ); } @@ -3879,20 +3880,20 @@ public void testMergedPostAggHavingSpec() { @Override public Sequence run( - Query query, Map responseContext + QueryPlus queryPlus, Map responseContext ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( - query.getResultOrdering(), + queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) ) ); } @@ -4923,12 +4924,6 @@ public boolean eval(Row row) { return (row.getFloatMetric("idx_subpostagg") < 3800); } - - @Override - public byte[] getCacheKey() - { - return new byte[0]; - } } ) .addOrderByColumn("alias") @@ -5194,12 +5189,6 @@ public boolean eval(Row row) { return (row.getFloatMetric("idx_subpostagg") < 3800); } - - @Override - public byte[] getCacheKey() - { - return new byte[0]; - } } ) .addOrderByColumn("alias") @@ -8390,4 +8379,620 @@ public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, outerQuery); TestHelper.assertExpectedObjects(expectedResults, results, ""); } + + @Test + public void testGroupByLimitPushDown() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.marketDimension, + "marketalias" + ) + ) + ) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "marketalias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 2 + ) + ) + .setAggregatorSpecs( + Lists.newArrayList( + QueryRunnerTestHelper.rowsCount + ) + ) + .setContext( + ImmutableMap.of( + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true + ) + ) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "marketalias", + "upfront", + "rows", + 186L + ), + GroupByQueryRunnerTestHelper.createExpectedRow( + "1970-01-01T00:00:00.000Z", + "marketalias", + "total_market", + "rows", + 186L + ) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "order-limit"); + } + + @Test + public void testMergeResultsWithLimitPushDown() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setContext( + ImmutableMap.of( + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true + ) + ) + .setGranularity(Granularities.ALL); + + final GroupByQuery allGranQuery = builder.build(); + + QueryRunner mergedRunner = factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + // simulate two daily segments + final Query query1 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) + ); + final Query query2 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) + ); + + return factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map responseContext) + { + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); + } + } + ).run(query, responseContext); + } + } + ); + Map context = Maps.newHashMap(); + List allGranExpectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 2L, "idx", 221L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L) + ); + + TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged"); + } + + @Test + public void testMergeResultsWithLimitPushDownSortByAgg() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "idx", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setContext( + ImmutableMap.of( + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true + ) + ) + .setGranularity(Granularities.ALL); + + final GroupByQuery allGranQuery = builder.build(); + + QueryRunner mergedRunner = factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + // simulate two daily segments + final Query query1 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) + ); + final Query query2 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) + ); + + return factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map responseContext) + { + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); + } + } + ).run(query, responseContext); + } + } + ); + Map context = Maps.newHashMap(); + + List allGranExpectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 6L, "idx", 4416L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 2L, "idx", 319L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) + ); + + Iterable results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.newArrayList()); + TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); + } + + @Test + public void testMergeResultsWithLimitPushDownSortByDimDim() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", "market") + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ), + new OrderByColumnSpec( + "market", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setContext( + ImmutableMap.of( + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true + ) + ) + .setGranularity(Granularities.ALL); + + final GroupByQuery allGranQuery = builder.build(); + + QueryRunner mergedRunner = factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + // simulate two daily segments + final Query query1 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) + ); + final Query query2 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) + ); + + return factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map responseContext) + { + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); + } + } + ).run(query, responseContext); + } + } + ); + Map context = Maps.newHashMap(); + + List allGranExpectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "market", "spot", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "market", "spot", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "upfront", "rows", 2L, "idx", 1817L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "total_market", "rows", 2L, "idx", 2342L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) + ); + + Iterable results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.newArrayList()); + TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); + } + + @Test + public void testMergeResultsWithLimitPushDownSortByDimAggDim() + { + if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + return; + } + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", "market") + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ), + new OrderByColumnSpec( + "idx", + OrderByColumnSpec.Direction.DESCENDING + ), + new OrderByColumnSpec( + "market", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setContext( + ImmutableMap.of( + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true + ) + ) + .setGranularity(Granularities.ALL); + + final GroupByQuery allGranQuery = builder.build(); + + QueryRunner mergedRunner = factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + // simulate two daily segments + final Query query1 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) + ); + final Query query2 = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) + ); + + return factory.getToolchest().mergeResults( + new QueryRunner() + { + @Override + public Sequence run(Query query, Map responseContext) + { + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext)) + ) + ); + } + } + ).run(query, responseContext); + } + } + ); + Map context = Maps.newHashMap(); + + List allGranExpectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "market", "spot", "rows", 2L, "idx", 243L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "market", "spot", "rows", 2L, "idx", 177L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "total_market", "rows", 2L, "idx", 2342L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "upfront", "rows", 2L, "idx", 1817L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) + ); + + Iterable results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.newArrayList()); + TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); + } + + @Test + public void testGroupByLimitPushDownPostAggNotSupported() + { + //if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("Limit push down when sorting by a post aggregator is not supported."); + //} + + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryRunnerTestHelper.allGran) + .setDimensions( + Arrays.asList( + new DefaultDimensionSpec( + QueryRunnerTestHelper.marketDimension, + "marketalias" + ) + ) + ) + .setInterval(QueryRunnerTestHelper.fullOnInterval) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "constant", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 2 + ) + ) + .setAggregatorSpecs( + Lists.newArrayList( + QueryRunnerTestHelper.rowsCount + ) + ) + .setPostAggregatorSpecs( + Lists.newArrayList( + new ConstantPostAggregator("constant", 1) + ) + ) + .setContext( + ImmutableMap.of( + GroupByQueryConfig.CTX_KEY_FORCE_LIMIT_PUSH_DOWN, + true + ) + ) + .build(); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + } + + @Test + public void testEmptySubqueryWithLimitPushDown() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + new DoubleMaxAggregatorFactory("idx", "idx") + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + null, + 5 + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Assert.assertFalse(results.iterator().hasNext()); + } + + + @Test + public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter( + "quality", + "function(dim){ return true; }", + null, + JavaScriptConfig.getEnabledInstance() + )) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 12 + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index"), + new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec( + new MultipleIntervalSegmentSpec( + ImmutableList.of( + new Interval("2011-04-01T00:00:00.000Z/2011-04-01T23:58:00.000Z"), + new Interval("2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z") + ) + ) + ) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setLimitSpec( + new DefaultLimitSpec( + Lists.newArrayList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 15 + ) + ) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), + + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L) + ); + + // Subqueries are handled by the ToolChest + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java index 0c6ea89544d5..b6f1551b20e3 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java @@ -21,8 +21,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.granularity.Granularities; import io.druid.query.Query; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.AggregatorFactory; @@ -34,6 +39,7 @@ import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.ordering.StringComparators; +import io.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -62,7 +68,11 @@ public void testQuerySerialization() throws IOException .setPostAggregatorSpecs(ImmutableList.of(new FieldAccessPostAggregator("x", "idx"))) .setLimitSpec( new DefaultLimitSpec( - ImmutableList.of(new OrderByColumnSpec("alias", OrderByColumnSpec.Direction.ASCENDING, StringComparators.LEXICOGRAPHIC)), + ImmutableList.of(new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.LEXICOGRAPHIC + )), 100 ) ) @@ -70,8 +80,27 @@ public void testQuerySerialization() throws IOException String json = jsonMapper.writeValueAsString(query); Query serdeQuery = jsonMapper.readValue(json, Query.class); - + Assert.assertEquals(query, serdeQuery); } + @Test + public void testRowOrderingMixTypes() + { + final GroupByQuery query = GroupByQuery.builder() + .setDataSource("dummy") + .setGranularity(Granularities.ALL) + .setInterval("2000/2001") + .addDimension(new DefaultDimensionSpec("foo", "foo", ValueType.LONG)) + .addDimension(new DefaultDimensionSpec("bar", "bar", ValueType.FLOAT)) + .addDimension(new DefaultDimensionSpec("baz", "baz", ValueType.STRING)) + .build(); + + final Ordering rowOrdering = query.getRowOrdering(false); + final int compare = rowOrdering.compare( + new MapBasedRow(0L, ImmutableMap.of("foo", 1, "bar", 1f, "baz", "a")), + new MapBasedRow(0L, ImmutableMap.of("foo", 1L, "bar", 1d, "baz", "b")) + ); + Assert.assertEquals(-1, compare); + } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 4ac25c7499b7..032cf0e7a20b 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -30,7 +30,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; @@ -76,9 +76,9 @@ public Object apply(final QueryRunner input) return new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - TimeseriesQuery tsQuery = (TimeseriesQuery) query; + TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery(); QueryRunner newRunner = factory.mergeRunners( MoreExecutors.sameThreadExecutor(), ImmutableList.>of(input) ); @@ -136,6 +136,7 @@ public GroupByTimeseriesQueryRunnerTest(QueryRunner runner) } // GroupBy handles timestamps differently when granularity is ALL + @Override @Test public void testFullOnTimeseriesMaxMin() { diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java index e6605400cec8..e7ca10b7171d 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java @@ -20,24 +20,34 @@ package io.druid.query.groupby.epinephelinae; import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.google.common.io.Files; import com.google.common.primitives.Ints; import io.druid.data.input.MapBasedRow; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel; import java.util.Comparator; import java.util.List; public class BufferGrouperTest { + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Test public void testSimple() { @@ -116,6 +126,34 @@ public void testGrowing() Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); } + @Test + public void testGrowing2() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final Grouper grouper = makeGrouper(columnSelectorFactory, 2_000_000_000, 2); + final int expectedMaxSize = 40988516; + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + for (int i = 0; i < expectedMaxSize; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i).isOk()); + } + Assert.assertFalse(grouper.aggregate(expectedMaxSize).isOk()); + } + + @Test + public void testGrowing3() + { + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final Grouper grouper = makeGrouper(columnSelectorFactory, Integer.MAX_VALUE, 2); + final int expectedMaxSize = 44938972; + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + for (int i = 0; i < expectedMaxSize; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i).isOk()); + } + Assert.assertFalse(grouper.aggregate(expectedMaxSize).isOk()); + } + @Test public void testNoGrowing() { @@ -144,14 +182,23 @@ public void testNoGrowing() Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); } - private static BufferGrouper makeGrouper( + private BufferGrouper makeGrouper( TestColumnSelectorFactory columnSelectorFactory, int bufferSize, int initialBuckets ) { + final MappedByteBuffer buffer; + + try { + buffer = Files.map(temporaryFolder.newFile(), FileChannel.MapMode.READ_WRITE, bufferSize); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + final BufferGrouper grouper = new BufferGrouper<>( - Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)), + Suppliers.ofInstance(buffer), GrouperTestUtil.intKeySerde(), columnSelectorFactory, new AggregatorFactory[]{ diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeapTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeapTest.java new file mode 100644 index 000000000000..336c1b2fc16a --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ByteBufferMinMaxOffsetHeapTest.java @@ -0,0 +1,261 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +public class ByteBufferMinMaxOffsetHeapTest +{ + @Test + public void testSimple() + { + int limit = 15; + ByteBuffer myBuffer = ByteBuffer.allocate(1000000); + ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.natural(), null); + + ArrayList values = Lists.newArrayList( + 30, 45, 81, 92, 68, 54, 66, 33, 89, 98, + 87, 62, 84, 39, 13, 32, 67, 50, 21, 53, + 93, 18, 86, 41, 14, 56, 51, 69, 91, 60, + 6, 2, 79, 4, 35, 17, 71, 22, 29, 76, + 57, 97, 73, 24, 94, 77, 80, 15, 52, 88, + 95, 96, 9, 3, 48, 58, 75, 82, 90, 65, + 36, 85, 20, 34, 37, 72, 11, 78, 28, 43, + 27, 12, 83, 38, 59, 19, 31, 46, 40, 63, + 23, 70, 26, 8, 64, 16, 10, 74, 7, 25, + 5, 42, 47, 44, 1, 49, 99 + ); + + for (int i = 0; i < values.size(); i++){ + heap.addOffset(values.get(i)); + } + + int x = heap.removeAt(8); + heap.addOffset(x); + + x = heap.removeAt(2); + heap.addOffset(x); + + Collections.sort(values); + List expected = values.subList(0, limit); + + List actual = Lists.newArrayList(); + for (int i = 0; i < limit; i++) { + int min = heap.removeMin(); + actual.add(min); + } + + Assert.assertEquals(expected, actual); + } + + + @Test + public void testRandom() + { + int limit = 20; + + Random rng = new Random(999); + + ArrayList values = Lists.newArrayList(); + for (int i = 0; i < 100000; i++) { + values.add(rng.nextInt(1000000)); + } + ArrayList deletedValues = Lists.newArrayList(); + + ByteBuffer myBuffer = ByteBuffer.allocate(1000000); + ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.natural(), null); + + for (int i = 0; i < values.size(); i++){ + int droppedOffset = heap.addOffset(values.get(i)); + Assert.assertTrue(heap.isIntact()); + + if (droppedOffset > 0) { + deletedValues.add(droppedOffset); + } + + // 15% chance to delete a random value for every two values added when heap is > 50% full + if (heap.getHeapSize() > (limit / 2) && i % 2 == 1) { + double deleteRoll = rng.nextDouble(); + if (deleteRoll > 0.15) { + int indexToRemove = rng.nextInt(heap.getHeapSize()); + int deadOffset = heap.removeAt(indexToRemove); + Assert.assertTrue(heap.isIntact()); + deletedValues.add(deadOffset); + } + } + } + + Collections.sort(values); + Collections.sort(deletedValues); + + for (int deletedValue : deletedValues) { + int idx = values.indexOf(deletedValue); + values.remove(idx); + } + + Assert.assertTrue(heap.getHeapSize() <= limit); + List expected = values.subList(0, heap.getHeapSize()); + + List actual = Lists.newArrayList(); + int initialHeapSize = heap.getHeapSize(); + for (int i = 0; i < initialHeapSize; i++){ + int min = heap.removeMin(); + actual.add(min); + } + + Assert.assertEquals(expected, actual); + } + + @Test + public void testRandom2() + { + int limit = 20000; + + Random rng = new Random(9999); + + ArrayList values = Lists.newArrayList(); + for (int i = 0; i < 100000; i++) { + values.add(rng.nextInt(1000000)); + } + ArrayList deletedValues = Lists.newArrayList(); + + ByteBuffer myBuffer = ByteBuffer.allocate(1000000); + ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.natural(), null); + + for (int i = 0; i < values.size(); i++){ + int droppedOffset = heap.addOffset(values.get(i)); + Assert.assertTrue(heap.isIntact()); + + if (droppedOffset > 0) { + deletedValues.add(droppedOffset); + } + + // 15% chance to delete a random value for every two values added when heap is > 50% full + if (heap.getHeapSize() > (limit / 2) && i % 2 == 1) { + double deleteRoll = rng.nextDouble(); + if (deleteRoll > 0.15) { + int indexToRemove = rng.nextInt(heap.getHeapSize()); + int deadOffset = heap.removeAt(indexToRemove); + Assert.assertTrue(heap.isIntact()); + deletedValues.add(deadOffset); + } + } + } + + Collections.sort(values); + Collections.sort(deletedValues); + + for (int deletedValue : deletedValues) { + int idx = values.indexOf(deletedValue); + values.remove(idx); + } + + Assert.assertTrue(heap.getHeapSize() <= limit); + List expected = values.subList(0, heap.getHeapSize()); + + List actual = Lists.newArrayList(); + int initialHeapSize = heap.getHeapSize(); + for (int i = 0; i < initialHeapSize; i++){ + int min = heap.removeMin(); + actual.add(min); + } + + Assert.assertEquals(expected, actual); + } + + + @Test + public void testRemove() + { + int limit = 100; + + ArrayList values = Lists.newArrayList( + 1, 20, 1000, 2, 3, 30, 40, 10, 11, 12, 13, 300, 400, 500, 600 + ); + + ByteBuffer myBuffer = ByteBuffer.allocate(1000000); + ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.natural(), null); + + for (int i = 0; i < values.size(); i++){ + heap.addOffset(values.get(i)); + Assert.assertTrue(heap.isIntact()); + } + + heap.removeOffset(12); + + Assert.assertTrue(heap.isIntact()); + + Collections.sort(values); + values.remove((Number) 12); + + List actual = Lists.newArrayList(); + for (int i = 0; i < values.size(); i++){ + int min = heap.removeMin(); + actual.add(min); + } + + Assert.assertEquals(values, actual); + } + + @Test + public void testRemove2() + { + int limit = 100; + + ArrayList values = Lists.newArrayList( + 1, 20, 1000, 2, 3, 30, 40, 10, 11, 12, 13, 300, 400, 500, 600, 4, 5, + 6, 7, 8, 9, 4, 5, 200, 250 + ); + + ByteBuffer myBuffer = ByteBuffer.allocate(1000000); + ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.natural(), null); + + for (int i = 0; i < values.size(); i++){ + heap.addOffset(values.get(i)); + } + Assert.assertTrue(heap.isIntact()); + + heap.removeOffset(2); + Assert.assertTrue(heap.isIntact()); + + Collections.sort(values); + values.remove((Number) 2); + Assert.assertTrue(heap.isIntact()); + + List actual = Lists.newArrayList(); + for (int i = 0; i < values.size(); i++){ + int min = heap.removeMin(); + actual.add(min); + } + + Assert.assertTrue(heap.isIntact()); + + Assert.assertEquals(values, actual); + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java index cdd064a2dd3a..c5fe90ad5550 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -25,7 +25,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.dimension.DimensionSpec; -import io.druid.query.groupby.epinephelinae.Grouper.KeyComparator; +import io.druid.query.groupby.epinephelinae.Grouper.BufferComparator; import io.druid.query.groupby.epinephelinae.Grouper.KeySerde; import io.druid.query.groupby.epinephelinae.Grouper.KeySerdeFactory; import io.druid.segment.ColumnSelectorFactory; @@ -107,9 +107,9 @@ public Long fromByteBuffer(ByteBuffer buffer, int position) } @Override - public KeyComparator bufferComparator() + public BufferComparator bufferComparator() { - return new KeyComparator() + return new BufferComparator() { @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) @@ -119,20 +119,29 @@ public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, }; } + @Override + public BufferComparator bufferComparatorWithAggregators( + AggregatorFactory[] aggregatorFactories, + int[] aggregatorOffsets + ) + { + return null; + } + @Override public void reset() {} }; } @Override - public Comparator objectComparator() + public Comparator> objectComparator(boolean forceDefaultOrder) { - return new Comparator() + return new Comparator>() { @Override - public int compare(Long o1, Long o2) + public int compare(Grouper.Entry o1, Grouper.Entry o2) { - return o1.compareTo(o2); + return o1.getKey().compareTo(o2.getKey()); } }; } @@ -184,7 +193,9 @@ public void testAggregate() throws InterruptedException, ExecutionException 1, null, null, - 8 + 8, + null, + false ); Future[] futures = new Future[8]; diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java index 5a3b4e829fbb..8f017caff953 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/IntKeySerde.java @@ -20,6 +20,7 @@ package io.druid.query.groupby.epinephelinae; import com.google.common.primitives.Ints; +import io.druid.query.aggregation.AggregatorFactory; import java.nio.ByteBuffer; import java.util.Comparator; @@ -33,7 +34,7 @@ private IntKeySerde() // No instantiation } - private static final Grouper.KeyComparator KEY_COMPARATOR = new Grouper.KeyComparator() + private static final Grouper.BufferComparator KEY_COMPARATOR = new Grouper.BufferComparator() { @Override public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) @@ -80,7 +81,15 @@ public Integer fromByteBuffer(ByteBuffer buffer, int position) } @Override - public Grouper.KeyComparator bufferComparator() + public Grouper.BufferComparator bufferComparator() + { + return KEY_COMPARATOR; + } + + @Override + public Grouper.BufferComparator bufferComparatorWithAggregators( + AggregatorFactory[] aggregatorFactories, int[] aggregatorOffsets + ) { return KEY_COMPARATOR; } diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java new file mode 100644 index 000000000000..7cd746379ee0 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/LimitedBufferGrouperTest.java @@ -0,0 +1,174 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.data.input.MapBasedRow; +import io.druid.java.util.common.IAE; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.nio.ByteBuffer; +import java.util.List; + +public class LimitedBufferGrouperTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testLimitAndBufferSwapping() + { + final int limit = 100; + final int keyBase = 100000; + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final LimitedBufferGrouper grouper = makeGrouper(columnSelectorFactory, 20000, 2, limit); + final int numRows = 1000; + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + for (int i = 0; i < numRows; i++) { + Assert.assertTrue(String.valueOf(i + keyBase), grouper.aggregate(i + keyBase).isOk()); + } + + // bucket size is hash(int) + key(int) + aggs(2 longs) + heap offset(int) = 28 bytes + // limit is 100 so heap occupies 101 * 4 bytes = 404 bytes + // buffer is 20000 bytes, so table arena size is 20000 - 404 = 19596 bytes + // table arena is split in halves when doing push down, so each half is 9798 bytes + // each table arena half can hold 9798 / 28 = 349 buckets, with load factor of 0.5 max buckets per half is 174 + // First buffer swap occurs when we hit 174 buckets + // Subsequent buffer swaps occur after every 74 buckets, since we keep 100 buckets due to the limit + // With 1000 keys inserted, this results in one swap at the first 174 buckets, then 11 swaps afterwards. + // After the last swap, we have 100 keys + 12 new keys inserted. + Assert.assertEquals(12, grouper.getGrowthCount()); + Assert.assertEquals(112, grouper.getSize()); + Assert.assertEquals(349, grouper.getBuckets()); + Assert.assertEquals(174, grouper.getMaxSize()); + Assert.assertEquals(100, grouper.getLimit()); + + // Aggregate slightly different row + // Since these keys are smaller, they will evict the previous 100 top entries + // First 100 of these new rows will be the expected results. + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 11L))); + for (int i = 0; i < numRows; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i).isOk()); + } + + // we added another 1000 unique keys + // previous size is 112, so next swap occurs after 62 rows + // after that, there are 1000 - 62 = 938 rows, 938 / 74 = 12 additional swaps after the first, + // with 50 keys being added after the final swap. + Assert.assertEquals(25, grouper.getGrowthCount()); + Assert.assertEquals(150, grouper.getSize()); + Assert.assertEquals(349, grouper.getBuckets()); + Assert.assertEquals(174, grouper.getMaxSize()); + Assert.assertEquals(100, grouper.getLimit()); + + final List> expected = Lists.newArrayList(); + for (int i = 0; i < limit; i++) { + expected.add(new Grouper.Entry<>(i, new Object[]{11L, 1L})); + } + + Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); + } + + @Test + public void testBufferTooSmall() + { + expectedException.expect(IAE.class); + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final LimitedBufferGrouper grouper = makeGrouper(columnSelectorFactory, 10, 2, 100); + } + + @Test + public void testMinBufferSize() + { + final int limit = 100; + final int keyBase = 100000; + final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); + final LimitedBufferGrouper grouper = makeGrouper(columnSelectorFactory, 11716, 2, limit); + final int numRows = 1000; + + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); + for (int i = 0; i < numRows; i++) { + Assert.assertTrue(String.valueOf(i + keyBase), grouper.aggregate(i + keyBase).isOk()); + } + + // With minimum buffer size, after the first swap, every new key added will result in a swap + Assert.assertEquals(899, grouper.getGrowthCount()); + Assert.assertEquals(101, grouper.getSize()); + Assert.assertEquals(202, grouper.getBuckets()); + Assert.assertEquals(101, grouper.getMaxSize()); + Assert.assertEquals(100, grouper.getLimit()); + + // Aggregate slightly different row + // Since these keys are smaller, they will evict the previous 100 top entries + // First 100 of these new rows will be the expected results. + columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 11L))); + for (int i = 0; i < numRows; i++) { + Assert.assertTrue(String.valueOf(i), grouper.aggregate(i).isOk()); + } + + Assert.assertEquals(1899, grouper.getGrowthCount()); + Assert.assertEquals(101, grouper.getSize()); + Assert.assertEquals(202, grouper.getBuckets()); + Assert.assertEquals(101, grouper.getMaxSize()); + Assert.assertEquals(100, grouper.getLimit()); + + final List> expected = Lists.newArrayList(); + for (int i = 0; i < limit; i++) { + expected.add(new Grouper.Entry<>(i, new Object[]{11L, 1L})); + } + + Assert.assertEquals(expected, Lists.newArrayList(grouper.iterator(true))); + } + + private static LimitedBufferGrouper makeGrouper( + TestColumnSelectorFactory columnSelectorFactory, + int bufferSize, + int initialBuckets, + int limit + ) + { + LimitedBufferGrouper grouper = new LimitedBufferGrouper<>( + Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)), + GrouperTestUtil.intKeySerde(), + columnSelectorFactory, + new AggregatorFactory[]{ + new LongSumAggregatorFactory("valueSum", "value"), + new CountAggregatorFactory("count") + }, + Integer.MAX_VALUE, + 0.5f, + initialBuckets, + limit, + false + ); + + grouper.init(); + return grouper; + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java b/processing/src/test/java/io/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java index 9c645def3e1a..06f16446eabf 100644 --- a/processing/src/test/java/io/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/having/DimensionSelectorHavingSpecTest.java @@ -20,20 +20,16 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.extraction.ExtractionFn; -import io.druid.query.extraction.IdentityExtractionFn; import io.druid.query.extraction.RegexDimExtractionFn; import org.junit.Assert; import org.junit.Test; -import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -107,15 +103,15 @@ public void testToString() String expected = "DimensionSelectorHavingSpec{" + "dimension='gender'," + " value='m'," + - " extractionFunction='regex(/^([^,]*),/, 1)'}"; - Assert.assertEquals(new DimensionSelectorHavingSpec("gender", "m", extractionFn).toString(), expected); + " extractionFn=regex(/^([^,]*),/, 1)}"; + Assert.assertEquals(expected, new DimensionSelectorHavingSpec("gender", "m", extractionFn).toString()); expected = "DimensionSelectorHavingSpec{" + "dimension='gender'," + " value='m'," + - " extractionFunction='Identity'}"; + " extractionFn=Identity}"; - Assert.assertEquals(new DimensionSelectorHavingSpec("gender", "m", null).toString(), expected); + Assert.assertEquals(expected, new DimensionSelectorHavingSpec("gender", "m", null).toString()); } @Test(expected = NullPointerException.class) @@ -158,47 +154,4 @@ public void testDimensionFilterSpec() assertTrue(spec.eval(getTestRow(ImmutableList.of("v1,v2", "none")))); } - - @Test - public void testGetCacheKey() - { - ExtractionFn extractionFn = IdentityExtractionFn.getInstance(); - byte[] dimBytes = "dimension".getBytes(Charsets.UTF_8); - byte[] valBytes = "v".getBytes(Charsets.UTF_8); - byte[] extFunKey = extractionFn.getCacheKey(); - - byte[] expected = ByteBuffer.allocate(3 + dimBytes.length + valBytes.length + extFunKey.length) - .put(CACHE_KEY) - .put(dimBytes) - .put(STRING_SEPARATOR) - .put(valBytes) - .put(STRING_SEPARATOR) - .put(extFunKey) - .array(); - - DimensionSelectorHavingSpec dfhs = new DimensionSelectorHavingSpec("dimension", "v", null); - DimensionSelectorHavingSpec dfhs1 = new DimensionSelectorHavingSpec("dimension", "v", null); - DimensionSelectorHavingSpec dfhs2 = new DimensionSelectorHavingSpec("dimensi", "onv", null); - - byte[] actual = dfhs.getCacheKey(); - - Assert.assertArrayEquals(expected, actual); - Assert.assertTrue(Arrays.equals(dfhs.getCacheKey(), dfhs1.getCacheKey())); - Assert.assertFalse(Arrays.equals(dfhs.getCacheKey(), dfhs2.getCacheKey())); - - extractionFn = new RegexDimExtractionFn("^([^,]*),", false, ""); - extFunKey = extractionFn.getCacheKey(); - dfhs = new DimensionSelectorHavingSpec("dimension", "v", extractionFn); - actual = dfhs.getCacheKey(); - expected = ByteBuffer.allocate(3 + dimBytes.length + valBytes.length + extFunKey.length) - .put(CACHE_KEY) - .put(dimBytes) - .put(STRING_SEPARATOR) - .put(valBytes) - .put(STRING_SEPARATOR) - .put(extFunKey) - .array(); - - Assert.assertArrayEquals(expected, actual); - } } diff --git a/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java index 044596937ad5..39543b48efce 100644 --- a/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java @@ -168,12 +168,6 @@ public boolean eval(Row row) counter.incrementAndGet(); return value; } - - @Override - public byte[] getCacheKey() - { - return new byte[0]; - } } @Test diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java index 58b939415a45..e248415f67ad 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorFactoryContainerTest.java @@ -110,6 +110,7 @@ public LookupExtractor get() return null; } + @Override public boolean equals(Object other) { return other instanceof TestLookupExtractorFactory; diff --git a/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java b/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java index 7adeb7c775d6..d9bc53f9e813 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupExtractorTest.java @@ -86,9 +86,9 @@ public void testApplyAllWithEmptySet() @Test public void testApplyAllWithNotExisting() { - Map actual = new HashMap<>(); - actual.put("not there", null); - Assert.assertEquals(actual, lookupExtractor.applyAll(Lists.newArrayList("not there"))); + Map expected = new HashMap<>(); + expected.put("not there", null); + Assert.assertEquals(expected, lookupExtractor.applyAll(Lists.newArrayList("not there"))); } @Test diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java index 8449f3e39d12..18ad069f0bc3 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java @@ -60,7 +60,7 @@ public void testCacheStrategy() throws Exception ); CacheStrategy strategy = - new SegmentMetadataQueryQueryToolChest(null).getCacheStrategy(query); + new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy(query); // Test cache key generation byte[] expectedKey = {0x04, 0x01, (byte) 0xFF, 0x00, 0x02, 0x04}; @@ -273,6 +273,7 @@ public void testMergeAggregatorsConflict() ); } + @SuppressWarnings("ArgumentParameterSwap") @Test public void testMergeRollup() { diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index fbb8e25cf57d..c2d7518b1cbc 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -143,8 +143,12 @@ public SegmentMetadataQueryTest( { final String id1 = differentIds ? "testSegment1" : "testSegment"; final String id2 = differentIds ? "testSegment2" : "testSegment"; - this.runner1 = mmap1 ? makeMMappedQueryRunner(id1, rollup1, FACTORY) : makeIncrementalIndexQueryRunner(id1, rollup1, FACTORY); - this.runner2 = mmap2 ? makeMMappedQueryRunner(id2, rollup2, FACTORY) : makeIncrementalIndexQueryRunner(id2, rollup2, FACTORY); + this.runner1 = mmap1 + ? makeMMappedQueryRunner(id1, rollup1, FACTORY) + : makeIncrementalIndexQueryRunner(id1, rollup1, FACTORY); + this.runner2 = mmap2 + ? makeMMappedQueryRunner(id2, rollup2, FACTORY) + : makeIncrementalIndexQueryRunner(id2, rollup2, FACTORY); this.mmap1 = mmap1; this.mmap2 = mmap2; this.rollup1 = rollup1; @@ -242,7 +246,7 @@ public SegmentMetadataQueryTest( null, null ) - // null_column will be included only for incremental index, which makes a little bigger result than expected + // null_column will be included only for incremental index, which makes a little bigger result than expected ), mmap2 ? 123969 : 124664, 1209, null, @@ -1092,12 +1096,56 @@ public void testCacheKeyWithListColumnIncluderator() .toInclude(new ListColumnIncluderator(Arrays.asList("fo", "o"))) .build(); - final byte[] oneColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(null).getCacheStrategy(oneColumnQuery) - .computeCacheKey(oneColumnQuery); + final byte[] oneColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy( + oneColumnQuery) + .computeCacheKey( + oneColumnQuery); - final byte[] twoColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(null).getCacheStrategy(twoColumnQuery) - .computeCacheKey(twoColumnQuery); + final byte[] twoColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy( + twoColumnQuery) + .computeCacheKey( + twoColumnQuery); Assert.assertFalse(Arrays.equals(oneColumnQueryCacheKey, twoColumnQueryCacheKey)); } + + @Test + public void testAnanlysisTypesBeingSet() + { + + SegmentMetadataQuery query1 = Druids.newSegmentMetadataQueryBuilder() + .dataSource("testing") + .toInclude(new ListColumnIncluderator(Arrays.asList("foo"))) + .build(); + + SegmentMetadataQuery query2 = Druids.newSegmentMetadataQueryBuilder() + .dataSource("testing") + .toInclude(new ListColumnIncluderator(Arrays.asList("foo"))) + .analysisTypes(SegmentMetadataQuery.AnalysisType.MINMAX) + .build(); + + SegmentMetadataQueryConfig emptyCfg = new SegmentMetadataQueryConfig(); + SegmentMetadataQueryConfig analysisCfg = new SegmentMetadataQueryConfig(); + analysisCfg.setDefaultAnalysisTypes(EnumSet.of(SegmentMetadataQuery.AnalysisType.CARDINALITY)); + + EnumSet analysis1 = query1.withFinalizedAnalysisTypes(emptyCfg) + .getAnalysisTypes(); + EnumSet analysis2 = query2.withFinalizedAnalysisTypes(emptyCfg) + .getAnalysisTypes(); + EnumSet analysisWCfg1 = query1.withFinalizedAnalysisTypes(analysisCfg) + .getAnalysisTypes(); + EnumSet analysisWCfg2 = query2.withFinalizedAnalysisTypes(analysisCfg) + .getAnalysisTypes(); + + EnumSet expectedAnalysis1 = new SegmentMetadataQueryConfig().getDefaultAnalysisTypes(); + EnumSet expectedAnalysis2 = EnumSet.of(SegmentMetadataQuery.AnalysisType.MINMAX); + EnumSet expectedAnalysisWCfg1 = EnumSet.of(SegmentMetadataQuery.AnalysisType.CARDINALITY); + EnumSet expectedAnalysisWCfg2 = EnumSet.of(SegmentMetadataQuery.AnalysisType.MINMAX); + + Assert.assertEquals(analysis1, expectedAnalysis1); + Assert.assertEquals(analysis2, expectedAnalysis2); + Assert.assertEquals(analysisWCfg1, expectedAnalysisWCfg1); + Assert.assertEquals(analysisWCfg2, expectedAnalysisWCfg2); + } + } diff --git a/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java b/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java index 189230710a0b..a7a7566feb64 100644 --- a/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java +++ b/processing/src/test/java/io/druid/query/monomorphicprocessing/StringRuntimeShapeTest.java @@ -34,6 +34,7 @@ static class Empty implements HotLoopCallee @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } } diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index 527a3f0fa662..1d23741d327a 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -30,6 +30,7 @@ import io.druid.js.JavaScriptConfig; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -168,16 +169,16 @@ public void testSearchWithCardinality() { @Override public Sequence> run( - Query> query, Map responseContext + QueryPlus> queryPlus, Map responseContext ) { - final Query> query1 = searchQuery.withQuerySegmentSpec( + final QueryPlus> queryPlus1 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-01-12/2011-02-28"))) ); - final Query> query2 = searchQuery.withQuerySegmentSpec( + final QueryPlus> queryPlus2 = queryPlus.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-03-01/2011-04-15"))) ); - return Sequences.concat(runner.run(query1, responseContext), runner.run(query2, responseContext)); + return Sequences.concat(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)); } } ); diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 04bf6d66dc82..26adab13fb8e 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -32,7 +32,7 @@ import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.query.Druids; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.Result; import io.druid.query.SegmentDescriptor; @@ -48,7 +48,7 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -68,7 +68,7 @@ public void testRetry() throws Exception new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return new Sequence() { @@ -151,10 +151,10 @@ public void testRetry2() throws Exception new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return Sequences.withEffect( - Sequences.simple(Arrays.asList(value)), + Sequences.simple(Collections.singletonList(value)), new Runnable() { @Override diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 69a1ea435154..b69072cd543b 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -25,7 +25,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; @@ -186,11 +186,11 @@ public void testUnionResultMerging() { @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map responseContext ) { - if (query.getDataSource().equals(new TableDataSource("ds1"))) { + if (queryPlus.getQuery().getDataSource().equals(new TableDataSource("ds1"))) { return Sequences.simple(descending ? Lists.reverse(ds1) : ds1); } else { return Sequences.simple(descending ? Lists.reverse(ds2) : ds2); diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index 9e59eb11cde3..0a063aeb59ab 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -27,7 +27,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.query.CacheStrategy; -import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -243,11 +243,11 @@ static class MockQueryRunner implements QueryRunner> @Override public Sequence> run( - Query> query, + QueryPlus> queryPlus, Map responseContext ) { - this.query = (TopNQuery) query; + this.query = (TopNQuery) queryPlus.getQuery(); return query.run(runner, responseContext); } } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 67dcbe3780ec..5bcc27c7ac12 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -28,9 +28,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Longs; import io.druid.collections.StupidPool; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; @@ -76,7 +79,6 @@ import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.ordering.StringComparators; import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.query.timeseries.TimeseriesQuery; import io.druid.segment.TestHelper; import io.druid.segment.column.Column; import io.druid.segment.column.ValueType; @@ -100,6 +102,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ @@ -111,13 +114,15 @@ public static Iterable constructorFeeder() throws IOException { List>> retVal = queryRunners(); List parameters = new ArrayList<>(); - for (int i = 0; i < 8; i++) { + for (int i = 0; i < 32; i++) { for (QueryRunner> firstParameter : retVal) { - Object[] params = new Object[4]; + Object[] params = new Object[6]; params[0] = firstParameter; params[1] = (i & 1) != 0; params[2] = (i & 2) != 0; params[3] = (i & 4) != 0; + params[4] = (i & 8) != 0; + params[5] = (i & 16) != 0; parameters.add(params); } } @@ -174,12 +179,20 @@ public TopNQueryRunnerTest( QueryRunner> runner, boolean specializeGeneric1AggPooledTopN, boolean specializeGeneric2AggPooledTopN, + boolean specializeHistorical1SimpleDoubleAggPooledTopN, + boolean specializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN, boolean duplicateSingleAggregatorQueries ) { this.runner = runner; - PooledTopNAlgorithm.specializeGeneric1AggPooledTopN = specializeGeneric1AggPooledTopN; - PooledTopNAlgorithm.specializeGeneric2AggPooledTopN = specializeGeneric2AggPooledTopN; + PooledTopNAlgorithm.setSpecializeGeneric1AggPooledTopN(specializeGeneric1AggPooledTopN); + PooledTopNAlgorithm.setSpecializeGeneric2AggPooledTopN(specializeGeneric2AggPooledTopN); + PooledTopNAlgorithm.setSpecializeHistorical1SimpleDoubleAggPooledTopN( + specializeHistorical1SimpleDoubleAggPooledTopN + ); + PooledTopNAlgorithm.setSpecializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN( + specializeHistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopN + ); this.duplicateSingleAggregatorQueries = duplicateSingleAggregatorQueries; } @@ -1886,19 +1899,7 @@ public void testTopNDimExtractionToOne() throws IOException .build(); Granularity gran = Granularities.DAY; - TimeseriesQuery tsQuery = Druids.newTimeseriesQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(gran) - .intervals(QueryRunnerTestHelper.fullOnInterval) - .aggregators( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexDoubleSum, - QueryRunnerTestHelper.qualityUniques - ) - ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build(); + List> expectedResults = Arrays.asList( new Result<>( new DateTime("2011-01-12T00:00:00.000Z"), @@ -4966,79 +4967,98 @@ public void testFullOnTopNDecorationOnNumeric() @Test public void testFullOnTopNWithAggsOnNumericDims() { - TopNQuery query = new TopNQueryBuilder() - .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryRunnerTestHelper.allGran) - .dimension(QueryRunnerTestHelper.marketDimension) - .metric(QueryRunnerTestHelper.indexMetric) - .threshold(4) - .intervals(QueryRunnerTestHelper.fullOnInterval) - .aggregators( - Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonAggregators, - Lists.newArrayList( - new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index"), - new LongSumAggregatorFactory("qlLong", "qualityLong"), - new DoubleSumAggregatorFactory("qlFloat", "qualityLong"), - new DoubleSumAggregatorFactory("qfFloat", "qualityFloat"), - new LongSumAggregatorFactory("qfLong", "qualityFloat") - ) - ) - ) - ) - .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) - .build(); - - List> expectedResults = Arrays.asList( - new Result( - new DateTime("2011-01-12T00:00:00.000Z"), - new TopNResultValue( - Arrays.>asList( - ImmutableMap.builder() - .put(QueryRunnerTestHelper.marketDimension, "total_market") - .put("rows", 186L) - .put("index", 215679.82879638672D) - .put("addRowsIndexConstant", 215866.82879638672D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1743.9217529296875D) - .put("minIndex", 792.3260498046875D) - .put("qlLong", 279000L) - .put("qlFloat", 279000.0) - .put("qfFloat", 2790000.0) - .put("qfLong", 2790000L) - .build(), - ImmutableMap.builder() - .put(QueryRunnerTestHelper.marketDimension, "upfront") - .put("rows", 186L) - .put("index", 192046.1060180664D) - .put("addRowsIndexConstant", 192233.1060180664D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_2) - .put("maxIndex", 1870.06103515625D) - .put("minIndex", 545.9906005859375D) - .put("qlLong", 279000L) - .put("qlFloat", 279000.0) - .put("qfFloat", 2790000.0) - .put("qfLong", 2790000L) - .build(), - ImmutableMap.builder() - .put(QueryRunnerTestHelper.marketDimension, "spot") - .put("rows", 837L) - .put("index", 95606.57232284546D) - .put("addRowsIndexConstant", 96444.57232284546D) - .put("uniques", QueryRunnerTestHelper.UNIQUES_9) - .put("maxIndex", 277.2735290527344D) - .put("minIndex", 59.02102279663086D) - .put("qlLong", 1171800L) - .put("qlFloat", 1171800.0) - .put("qfFloat", 11718000.0) - .put("qfLong", 11718000L) - .build() - ) - ) - ) + List>> aggregations = new ArrayList<>(); + aggregations.add(new Pair<>( + QueryRunnerTestHelper.rowsCount, + Longs.asList(186L, 186L, 837L) + )); + Pair> indexAggregation = new Pair<>( + QueryRunnerTestHelper.indexDoubleSum, + Doubles.asList(215679.82879638672D, 192046.1060180664D, 95606.57232284546D) ); - assertExpectedResults(expectedResults, query); + aggregations.add(indexAggregation); + aggregations.add(new Pair<>( + QueryRunnerTestHelper.qualityUniques, + Doubles.asList(QueryRunnerTestHelper.UNIQUES_2, QueryRunnerTestHelper.UNIQUES_2, QueryRunnerTestHelper.UNIQUES_9) + )); + aggregations.add(new Pair<>( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + Doubles.asList(1743.9217529296875D, 1870.06103515625D, 277.2735290527344D) + )); + aggregations.add(new Pair<>( + new DoubleMinAggregatorFactory("minIndex", "index"), + Doubles.asList(792.3260498046875D, 545.9906005859375D, 59.02102279663086D) + )); + aggregations.add(new Pair<>( + new LongSumAggregatorFactory("qlLong", "qualityLong"), + Longs.asList(279000L, 279000L, 1171800L) + )); + aggregations.add(new Pair<>( + new DoubleSumAggregatorFactory("qlFloat", "qualityLong"), + Doubles.asList(279000.0, 279000.0, 1171800.0) + )); + aggregations.add(new Pair<>( + new DoubleSumAggregatorFactory("qfFloat", "qualityFloat"), + Doubles.asList(2790000.0, 2790000.0, 11718000.0) + )); + aggregations.add(new Pair<>( + new LongSumAggregatorFactory("qfLong", "qualityFloat"), + Longs.asList(2790000L, 2790000L, 11718000L) + )); + + List>>> aggregationCombinations = new ArrayList<>(); + for (Pair> aggregation : aggregations) { + aggregationCombinations.add(Collections.singletonList(aggregation)); + } + aggregationCombinations.add(aggregations); + + for (List>> aggregationCombination : aggregationCombinations) { + boolean hasIndexAggregator = aggregationCombination.stream().anyMatch(agg -> "index".equals(agg.lhs.getName())); + boolean hasRowsAggregator = aggregationCombination.stream().anyMatch(agg -> "rows".equals(agg.lhs.getName())); + TopNQueryBuilder queryBuilder = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(QueryRunnerTestHelper.marketDimension) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators(aggregationCombination.stream().map(agg -> agg.lhs).collect(Collectors.toList())); + String metric; + if (hasIndexAggregator) { + metric = "index"; + } else { + metric = aggregationCombination.get(0).lhs.getName(); + } + queryBuilder.metric(metric); + if (hasIndexAggregator && hasRowsAggregator) { + queryBuilder.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)); + } + TopNQuery query = queryBuilder.build(); + + ImmutableMap.Builder row1 = ImmutableMap.builder() + .put(QueryRunnerTestHelper.marketDimension, "total_market"); + ImmutableMap.Builder row2 = ImmutableMap.builder() + .put(QueryRunnerTestHelper.marketDimension, "upfront"); + ImmutableMap.Builder row3 = ImmutableMap.builder() + .put(QueryRunnerTestHelper.marketDimension, "spot"); + if (hasIndexAggregator && hasRowsAggregator) { + row1.put("addRowsIndexConstant", 215866.82879638672D); + row2.put("addRowsIndexConstant", 192233.1060180664D); + row3.put("addRowsIndexConstant", 96444.57232284546D); + } + aggregationCombination.forEach(agg -> { + row1.put(agg.lhs.getName(), agg.rhs.get(0)); + row2.put(agg.lhs.getName(), agg.rhs.get(1)); + row3.put(agg.lhs.getName(), agg.rhs.get(2)); + }); + List> rows = Lists.newArrayList(row1.build(), row2.build(), row3.build()); + rows.sort((r1, r2) -> ((Comparable) r2.get(metric)).compareTo(r1.get(metric))); + List> expectedResults = Collections.singletonList( + new Result<>( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue(rows) + ) + ); + assertExpectedResults(expectedResults, query); + } } } diff --git a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java index a09dd68e0884..717c00a7c38a 100644 --- a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java +++ b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java @@ -36,6 +36,7 @@ public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexedSupplierTest { + @Override @Before public void setUpSimple(){ vals = Arrays.asList( @@ -61,6 +62,7 @@ public IndexedInts apply(int[] input) ); } + @Override @After public void teardown(){ indexedSupplier = null; diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java index bffceb2c1fb3..fc629594bc21 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -201,7 +201,6 @@ public void testPersistWithSegmentMetadata() throws IOException finally { if (index != null) { index.close(); - ; } if (outDir != null) { diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java index 03df1951f9ce..cd1f98115dea 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java @@ -1352,6 +1352,7 @@ public void testDifferentMetrics() ); } + @SuppressWarnings("ArgumentParameterSwap") private List> getIndexes(int index1, int index2) { return Arrays.asList( diff --git a/processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java b/processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java index 3d53ac0effc4..8d6bcf45fcb3 100644 --- a/processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java +++ b/processing/src/test/java/io/druid/segment/TestFloatColumnSelector.java @@ -23,11 +23,9 @@ public abstract class TestFloatColumnSelector implements FloatColumnSelector { - /** - * Don't care about runtime shape in tests - */ @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } } diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 6f5297684919..7a29b9248604 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -291,7 +291,9 @@ public static IncrementalIndex loadIncrementalIndex( new DimensionsSpec(DIMENSION_SCHEMAS, null, null), "\t", "\u0001", - Arrays.asList(COLUMNS) + Arrays.asList(COLUMNS), + false, + 0 ) , "utf8" ); diff --git a/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java b/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java index f4b5a32f2585..6c6db1da47a0 100644 --- a/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java +++ b/processing/src/test/java/io/druid/segment/TestLongColumnSelector.java @@ -23,11 +23,9 @@ public abstract class TestLongColumnSelector implements LongColumnSelector { - /** - * Don't care about runtime shape in tests - */ @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java index d47b060b1f0a..a1e681b75d18 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java @@ -22,6 +22,7 @@ import com.google.common.base.Supplier; import com.google.common.io.ByteSink; import com.google.common.primitives.Floats; +import com.google.common.primitives.Ints; import io.druid.java.util.common.guava.CloseQuietly; import org.junit.Assert; import org.junit.Test; @@ -35,7 +36,6 @@ import java.nio.ByteOrder; import java.nio.channels.Channels; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -167,7 +167,7 @@ private void assertIndexMatchesVals(IndexedFloats indexed, float[] vals) indices[i] = i; } - Collections.shuffle(Arrays.asList(indices)); + Collections.shuffle(Ints.asList(indices)); // random access for (int i = 0; i < indexed.size(); ++i) { int k = indices[i]; @@ -224,7 +224,7 @@ public void run() final float indexedVal = indexed.get(j); if (Floats.compare(val, indexedVal) != 0) { failureHappened.set(true); - reason.set(String.format("Thread1[%d]: %d != %d", j, val, indexedVal)); + reason.set(String.format("Thread1[%d]: %f != %f", j, val, indexedVal)); stopLatch.countDown(); return; } @@ -263,7 +263,7 @@ public void run() final float indexedVal = indexed2.get(j); if (Floats.compare(val, indexedVal) != 0) { failureHappened.set(true); - reason.set(String.format("Thread2[%d]: %d != %d", j, val, indexedVal)); + reason.set(String.format("Thread2[%d]: %f != %f", j, val, indexedVal)); stopLatch.countDown(); return; } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java index 66b6dfa94353..7968fc2b071b 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java @@ -34,7 +34,6 @@ import java.nio.ByteOrder; import java.nio.IntBuffer; import java.nio.channels.Channels; -import java.util.Arrays; import java.util.Collections; import java.util.Random; import java.util.concurrent.CountDownLatch; @@ -331,7 +330,7 @@ private void assertIndexMatchesVals() indices[i] = i; } - Collections.shuffle(Arrays.asList(indices)); + Collections.shuffle(Ints.asList(indices)); // random access for (int i = 0; i < indexed.size(); ++i) { int k = indices[i]; diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java index ffae76ff6535..076a0a7fb9e5 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java @@ -21,6 +21,7 @@ import com.google.common.base.Supplier; import com.google.common.io.ByteSink; +import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.java.util.common.guava.CloseQuietly; import org.junit.Assert; @@ -35,7 +36,6 @@ import java.nio.ByteOrder; import java.nio.channels.Channels; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -189,7 +189,7 @@ private void assertIndexMatchesVals(IndexedLongs indexed, long[] vals) indices[i] = i; } - Collections.shuffle(Arrays.asList(indices)); + Collections.shuffle(Ints.asList(indices)); // random access for (int i = 0; i < indexed.size(); ++i) { int k = indices[i]; diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java index c14942367aef..b3aa68d4974d 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -115,10 +115,9 @@ private void generateVals(final int totalSize, final int maxValue) throws IOExce private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception { FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); - final IOPeon ioPeon = new TmpFileIOPeon(); final IndexedMultivalue indexedMultivalue; - try { + try (IOPeon ioPeon = new TmpFileIOPeon()) { int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy @@ -170,9 +169,6 @@ public IndexedInts apply(@Nullable final int[] input) } CloseQuietly.close(indexedMultivalue); } - finally { - ioPeon.close(); - } } int getMaxValue(final List vals) @@ -245,10 +241,9 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF offsetChunkFactor )).toFile(); FileSmoosher smoosher = new FileSmoosher(tmpDirectory); - final IOPeon ioPeon = new TmpFileIOPeon(); int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; - try { + try (IOPeon ioPeon = new TmpFileIOPeon()) { CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( offsetChunkFactor, compressionStrategy, @@ -316,9 +311,6 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF CloseQuietly.close(indexedMultivalue); mapper.close(); } - finally { - ioPeon.close(); - } } @Test diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java index bb29c04513ca..5080556b46c4 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java @@ -38,7 +38,6 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.Channels; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Random; @@ -361,7 +360,7 @@ private void assertIndexMatchesVals() indices[i] = i; } - Collections.shuffle(Arrays.asList(indices)); + Collections.shuffle(Ints.asList(indices)); // random access for (int i = 0; i < indexed.size(); ++i) { int k = indices[i]; diff --git a/processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java b/processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java index 3e8e7398f9fe..ef284067997b 100644 --- a/processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java +++ b/processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java @@ -42,7 +42,8 @@ public static Collection constructorFeeder() throws IOException { return Arrays.asList( new Object[][]{ - {VSizeIndexedInts.fromArray(array)} + {VSizeIndexedInts.fromArray(array)}, + {ArrayBasedIndexedInts.of(array)} } ); } diff --git a/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java index 75610d2eff0e..fa442bee5969 100644 --- a/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/io/druid/segment/filter/FilterPartitionTest.java @@ -61,7 +61,7 @@ @RunWith(Parameterized.class) public class FilterPartitionTest extends BaseFilterTest { - private class NoBitmapSelectorFilter extends SelectorFilter + private static class NoBitmapSelectorFilter extends SelectorFilter { public NoBitmapSelectorFilter( String dimension, @@ -78,7 +78,7 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector) } } - private class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter + private static class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter { public NoBitmapDimensionPredicateFilter( final String dimension, @@ -96,7 +96,7 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector) } } - private class NoBitmapSelectorDimFilter extends SelectorDimFilter + private static class NoBitmapSelectorDimFilter extends SelectorDimFilter { public NoBitmapSelectorDimFilter( String dimension, diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index d2819d834751..53d26b4526da 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -169,11 +169,11 @@ protected Integer addToFacts( // Last ditch sanity checks - if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == null) { + if (numEntries.get() >= maxRowCount && getFacts().getPriorIndex(key) == TimeAndDims.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows reached"); } - final Integer prev = getFacts().putIfAbsent(key, rowIndex); - if (null == prev) { + final int prev = getFacts().putIfAbsent(key, rowIndex); + if (TimeAndDims.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { // We lost a race diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java index 10b30bde40df..b788de84a1a5 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexTest.java @@ -31,7 +31,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Map; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; @@ -83,8 +82,8 @@ public void run() public void run() { while (!Thread.interrupted()) { - for (Map.Entry row : index.getFacts().entrySet()) { - if (index.getMetricLongValue(row.getValue(), 0) != 1) { + for (IncrementalIndex.TimeAndDims row : index.getFacts().keySet()) { + if (index.getMetricLongValue(row.getRowIndex(), 0) != 1) { checkFailedCount.addAndGet(1); } } diff --git a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java index 7b8f8c8abf32..7235ccc5b2a0 100644 --- a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java @@ -46,15 +46,13 @@ import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; -import it.unimi.dsi.fastutil.ints.IntIterator; -import it.unimi.dsi.fastutil.ints.IntIterators; +import io.druid.segment.data.ZeroIndexedInts; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import javax.annotation.Nullable; -import java.io.IOException; import java.util.Arrays; import java.util.List; @@ -280,38 +278,7 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, Colu @Override public IndexedInts getRow() { - return new IndexedInts() - { - @Override - public int size() - { - return 1; - } - - @Override - public int get(int index) - { - return 0; - } - - @Override - public IntIterator iterator() - { - return IntIterators.singleton(0); - } - - @Override - public void fill(int index, int[] toFill) - { - throw new UnsupportedOperationException("fill not supported"); - } - - @Override - public void close() throws IOException - { - - } - }; + return ZeroIndexedInts.instance(); } @Override @@ -362,6 +329,7 @@ public int lookupId(final String name) @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { + // Don't care about runtime shape in tests } }; diff --git a/processing/src/test/resources/ProtoTest.proto b/processing/src/test/resources/ProtoTest.proto deleted file mode 100644 index 956db5259ac9..000000000000 --- a/processing/src/test/resources/ProtoTest.proto +++ /dev/null @@ -1,31 +0,0 @@ - -package prototest; -option java_package = "io.druid.indexer.data"; -option java_outer_classname = "ProtoTestEventWrapper"; - - - -message ProtoTestEvent { - - -enum EventCategory { - CATEGORY_ZERO = 0; - CATEGORY_ONE = 1; - CATEGORY_TWO = 2; -} - - required EventCategory eventType = 1; - - required uint64 id = 2; - required string timestamp = 3; - optional uint32 someOtherId = 4; - optional bool isValid = 5; - optional string description = 6; - - optional float someFloatColumn = 7; - optional uint32 someIntColumn = 8; - optional uint64 someLongColumn = 9; - - - -} diff --git a/processing/src/test/resources/prototest.desc b/processing/src/test/resources/prototest.desc deleted file mode 100644 index 649ce5bcb8fe5df743b9fff73e1835e4467efd4e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 400 zcmY+APfNov7>8|F-TG?(EHdpy7`wPr4jw#sbET7kSQ*8k7YR$4fu<>GCfkSb>-hOh zst$U}%k%p^;Z5MH2hp>VS{AI)xV97W+d~^L`F0^Z#MvjSG`a(y*`e6iEOEx}G`zX~ z6|I50ZQMF3(QG5t9s#(VCa#e@Gz4FBVH6_jq-T(i&f&C?g5~;ysRd2Eu`>-&1byDT zkkW|91`KSAvSzJ#t+}j{69av836WLjr6f9)1S^I~`K2^4Oi-!qr*|ESP!T4pX(| X6!VI*9aESRO=hF+N39g8Yo>kxsG4&{ diff --git a/processing/src/test/resources/test.runtime.properties b/processing/src/test/resources/test.runtime.properties index d84cf995ee1f..f93e3398ac7c 100644 --- a/processing/src/test/resources/test.runtime.properties +++ b/processing/src/test/resources/test.runtime.properties @@ -8,3 +8,5 @@ druid.metadata.storage.tables.taskLock=fff_tasklock druid.metadata.storage.tables.audit=ggg_audit druid.metadata.storage.tables.dataSource=hhh_dataSource druid.metadata.storage.tables.supervisors=iii_supervisors +druid.query.segmentMetadata.defaultAnalysisTypes=["cardinality", "size"] +druid.query.segmentMetadata.defaultHistory=P2W diff --git a/server/pom.xml b/server/pom.xml index a6898e8bcf6b..4f263448ab37 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -181,6 +181,10 @@ org.apache.commons commons-math3 + + it.unimi.dsi + fastutil + diff --git a/server/src/main/java/io/druid/client/AbstractCuratorServerInventoryView.java b/server/src/main/java/io/druid/client/AbstractCuratorServerInventoryView.java new file mode 100644 index 000000000000..996bf559c5e1 --- /dev/null +++ b/server/src/main/java/io/druid/client/AbstractCuratorServerInventoryView.java @@ -0,0 +1,365 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.MapMaker; +import com.metamx.emitter.EmittingLogger; +import io.druid.concurrent.Execs; +import io.druid.curator.inventory.CuratorInventoryManager; +import io.druid.curator.inventory.CuratorInventoryManagerStrategy; +import io.druid.curator.inventory.InventoryManagerConfig; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.timeline.DataSegment; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.utils.ZKPaths; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + */ +public abstract class AbstractCuratorServerInventoryView implements ServerInventoryView +{ + + private final EmittingLogger log; + private final CuratorFramework curator; + private final CuratorInventoryManager inventoryManager; + private final AtomicBoolean started = new AtomicBoolean(false); + + private final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); + private final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); + + public AbstractCuratorServerInventoryView( + final EmittingLogger log, + final String announcementsPath, + final String inventoryPath, + final CuratorFramework curator, + final ObjectMapper jsonMapper, + final TypeReference typeReference + ) + { + this.log = log; + this.curator = curator; + this.inventoryManager = new CuratorInventoryManager<>( + curator, + new InventoryManagerConfig() + { + @Override + public String getContainerPath() + { + return announcementsPath; + } + + @Override + public String getInventoryPath() + { + return inventoryPath; + } + }, + Execs.singleThreaded("ServerInventoryView-%s"), + new CuratorInventoryManagerStrategy() + { + @Override + public DruidServer deserializeContainer(byte[] bytes) + { + try { + return jsonMapper.readValue(bytes, DruidServer.class); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + @Override + public InventoryType deserializeInventory(byte[] bytes) + { + try { + return jsonMapper.readValue(bytes, typeReference); + } + catch (IOException e) { + CharBuffer.wrap(StringUtils.fromUtf8(bytes).toCharArray()); + CharBuffer charBuffer = Charsets.UTF_8.decode(ByteBuffer.wrap(bytes)); + log.error(e, "Could not parse json: %s", charBuffer.toString()); + throw Throwables.propagate(e); + } + } + + @Override + public void newContainer(DruidServer container) + { + log.info("New Server[%s]", container); + } + + @Override + public void deadContainer(DruidServer deadContainer) + { + log.info("Server Disappeared[%s]", deadContainer); + runServerCallbacks(deadContainer); + } + + @Override + public DruidServer updateContainer(DruidServer oldContainer, DruidServer newContainer) + { + return newContainer.addDataSegments(oldContainer); + } + + @Override + public DruidServer addInventory( + final DruidServer container, + String inventoryKey, + final InventoryType inventory + ) + { + return addInnerInventory(container, inventoryKey, inventory); + } + + @Override + public DruidServer updateInventory( + DruidServer container, String inventoryKey, InventoryType inventory + ) + { + return updateInnerInventory(container, inventoryKey, inventory); + } + + @Override + public DruidServer removeInventory(final DruidServer container, String inventoryKey) + { + return removeInnerInventory(container, inventoryKey); + } + + @Override + public void inventoryInitialized() + { + log.info("Inventory Initialized"); + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentViewInitialized(); + } + } + ); + } + } + ); + } + + @LifecycleStart + public void start() throws Exception + { + synchronized (started) { + if (!started.get()) { + inventoryManager.start(); + started.set(true); + } + } + } + + @LifecycleStop + public void stop() throws IOException + { + synchronized (started) { + if (started.getAndSet(false)) { + inventoryManager.stop(); + } + } + } + + @Override + public boolean isStarted() + { + return started.get(); + } + + @Override + public DruidServer getInventoryValue(String containerKey) + { + return inventoryManager.getInventoryValue(containerKey); + } + + @Override + public Iterable getInventory() + { + return inventoryManager.getInventory(); + } + + @Override + public void registerServerCallback(Executor exec, ServerCallback callback) + { + serverCallbacks.put(callback, exec); + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + segmentCallbacks.put(callback, exec); + } + + public InventoryManagerConfig getInventoryManagerConfig() + { + return inventoryManager.getConfig(); + } + + protected void runSegmentCallbacks( + final Function fn + ) + { + for (final Map.Entry entry : segmentCallbacks.entrySet()) { + entry.getValue().execute( + new Runnable() + { + @Override + public void run() + { + if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { + segmentCallbackRemoved(entry.getKey()); + segmentCallbacks.remove(entry.getKey()); + } + } + } + ); + } + } + + protected void runServerCallbacks(final DruidServer server) + { + for (final Map.Entry entry : serverCallbacks.entrySet()) { + entry.getValue().execute( + new Runnable() + { + @Override + public void run() + { + if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) { + serverCallbacks.remove(entry.getKey()); + } + } + } + ); + } + } + + protected void addSingleInventory( + final DruidServer container, + final DataSegment inventory + ) + { + log.debug("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); + + if (container.getSegment(inventory.getIdentifier()) != null) { + log.warn( + "Not adding or running callbacks for existing segment[%s] on server[%s]", + inventory.getIdentifier(), + container.getName() + ); + + return; + } + + container.addDataSegment(inventory.getIdentifier(), inventory); + + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentAdded(container.getMetadata(), inventory); + } + } + ); + } + + protected void removeSingleInventory(final DruidServer container, String inventoryKey) + { + log.debug("Server[%s] removed segment[%s]", container.getName(), inventoryKey); + final DataSegment segment = container.getSegment(inventoryKey); + + if (segment == null) { + log.warn( + "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]", + inventoryKey, + container.getName() + ); + + return; + } + + container.removeDataSegment(inventoryKey); + + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentRemoved(container.getMetadata(), segment); + } + } + ); + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + try { + String toServedSegPath = ZKPaths.makePath( + ZKPaths.makePath(getInventoryManagerConfig().getInventoryPath(), serverKey), + segment.getIdentifier() + ); + return curator.checkExists().forPath(toServedSegPath) != null; + } catch (Exception ex) { + throw Throwables.propagate(ex); + } + } + + protected abstract DruidServer addInnerInventory( + final DruidServer container, + String inventoryKey, + final InventoryType inventory + ); + + protected abstract DruidServer updateInnerInventory( + final DruidServer container, + String inventoryKey, + final InventoryType inventory + ); + + protected abstract DruidServer removeInnerInventory( + final DruidServer container, + String inventoryKey + ); + + protected abstract void segmentCallbackRemoved(SegmentCallback callback); +} diff --git a/server/src/main/java/io/druid/client/BatchServerInventoryView.java b/server/src/main/java/io/druid/client/BatchServerInventoryView.java index c6a6352af1f1..9fd9f5888334 100644 --- a/server/src/main/java/io/druid/client/BatchServerInventoryView.java +++ b/server/src/main/java/io/druid/client/BatchServerInventoryView.java @@ -45,7 +45,7 @@ /** */ @ManageLifecycle -public class BatchServerInventoryView extends ServerInventoryView> +public class BatchServerInventoryView extends AbstractCuratorServerInventoryView> implements FilteredServerInventoryView { private static final EmittingLogger log = new EmittingLogger(BatchServerInventoryView.class); @@ -169,6 +169,7 @@ protected DruidServer removeInnerInventory(final DruidServer container, String i return container; } + @Override public void registerSegmentCallback( final Executor exec, final SegmentCallback callback, diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index c86123b0ec85..3256dc370b76 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -59,6 +59,7 @@ import io.druid.query.CacheStrategy; import io.druid.query.Query; import io.druid.query.QueryContexts; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -139,8 +140,9 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data } @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { + final Query query = queryPlus.getQuery(); final QueryToolChest> toolChest = warehouse.getToolChest(query); final CacheStrategy> strategy = toolChest.getCacheStrategy(query); @@ -270,7 +272,7 @@ public ShardSpec apply(PartitionChunk input) Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; for (Pair p : segments) { - if (!p.lhs.pick().getServer().isAssignable()) { + if (!p.lhs.pick().getServer().segmentReplicatable()) { hasOnlyHistoricalSegments = false; break; } @@ -427,19 +429,14 @@ private void addSequencesFromServer(ArrayList> listOfSequences) final MultipleSpecificSegmentSpec segmentSpec = new MultipleSpecificSegmentSpec(descriptors); final Sequence resultSeqToAdd; - if (!server.isAssignable() || !populateCache || isBySegment) { // Direct server queryable + if (!server.segmentReplicatable() || !populateCache || isBySegment) { // Direct server queryable if (!isBySegment) { - resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), responseContext); + resultSeqToAdd = clientQueryable.run(queryPlus.withQuerySegmentSpec(segmentSpec), responseContext); } else { // bySegment queries need to be de-serialized, see DirectDruidClient.run() - - @SuppressWarnings("unchecked") - final Query>> bySegmentQuery = - (Query>>) ((Query) query); - @SuppressWarnings("unchecked") final Sequence>> resultSequence = clientQueryable.run( - bySegmentQuery.withQuerySegmentSpec(segmentSpec), + queryPlus.withQuerySegmentSpec(segmentSpec), responseContext ); @@ -472,7 +469,7 @@ public Result> apply(Result>> runningSequence = clientQueryable.run( - rewrittenQuery.withQuerySegmentSpec(segmentSpec), + queryPlus.withQuery(rewrittenQuery.withQuerySegmentSpec(segmentSpec)), responseContext ); resultSeqToAdd = new MergeSequence( diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index 418098c9d55f..17df72a59967 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -38,6 +38,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.CacheStrategy; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; @@ -83,8 +84,9 @@ public CachingQueryRunner( } @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { + Query query = queryPlus.getQuery(); final CacheStrategy strategy = toolChest.getCacheStrategy(query); final boolean populateCache = CacheUtil.populateCacheOnDataNodes(query, strategy, cacheConfig); final boolean useCache = CacheUtil.useCacheOnDataNodes(query, strategy, cacheConfig); @@ -145,7 +147,7 @@ public void cleanup(Iterator iterFromMake) return Sequences.withEffect( Sequences.map( - base.run(query, responseContext), + base.run(queryPlus, responseContext), new Function() { @Override @@ -190,7 +192,7 @@ public void run() backgroundExecutorService ); } else { - return base.run(query, responseContext); + return base.run(queryPlus, responseContext); } } diff --git a/server/src/main/java/io/druid/client/CoordinatorServerView.java b/server/src/main/java/io/druid/client/CoordinatorServerView.java index ee32b07c1db4..a572deecd14a 100644 --- a/server/src/main/java/io/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/io/druid/client/CoordinatorServerView.java @@ -207,4 +207,16 @@ public Iterable getInventory() { return baseView.getInventory(); } + + @Override + public boolean isStarted() + { + return baseView.isStarted(); + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + return baseView.isSegmentLoadedByServer(serverKey, segment); + } } diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index c3311798a5a6..3ab0c3f137f1 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -30,6 +30,7 @@ import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.base.Charsets; import com.google.common.base.Throwables; +import com.google.common.collect.MapMaker; import com.google.common.collect.Maps; import com.google.common.io.ByteSource; import com.google.common.util.concurrent.FutureCallback; @@ -42,6 +43,7 @@ import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.common.utils.StringUtils; import io.druid.java.util.common.IAE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.RE; @@ -55,18 +57,22 @@ import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; +import io.druid.query.ResourceLimitExceededException; import io.druid.query.Result; import io.druid.query.aggregation.MetricManipulatorFns; +import io.druid.server.initialization.ServerConfig; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBufferInputStream; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponse; +import org.joda.time.Duration; import javax.ws.rs.core.MediaType; import java.io.Closeable; @@ -83,14 +89,19 @@ import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; /** */ public class DirectDruidClient implements QueryRunner { + public static final String QUERY_FAIL_TIME = "queryFailTime"; + public static final String QUERY_TOTAL_BYTES_GATHERED = "queryTotalBytesGathered"; + private static final Logger log = new Logger(DirectDruidClient.class); private static final Map, Pair> typesMap = Maps.newConcurrentMap(); @@ -105,6 +116,31 @@ public class DirectDruidClient implements QueryRunner private final AtomicInteger openConnections; private final boolean isSmile; + public static > QueryType withDefaultTimeoutAndMaxScatterGatherBytes(final QueryType query, ServerConfig serverConfig) + { + return (QueryType) QueryContexts.withMaxScatterGatherBytes( + QueryContexts.withDefaultTimeout( + (Query) query, + serverConfig.getDefaultQueryTimeout() + ), + serverConfig.getMaxScatterGatherBytes() + ); + } + + public static Map makeResponseContextForQuery(Query query, long startTimeMillis) + { + final Map responseContext = new MapMaker().makeMap(); + responseContext.put( + DirectDruidClient.QUERY_FAIL_TIME, + startTimeMillis + QueryContexts.getTimeout(query) + ); + responseContext.put( + DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED, + new AtomicLong() + ); + return responseContext; + } + public DirectDruidClient( QueryToolChestWarehouse warehouse, QueryWatcher queryWatcher, @@ -131,8 +167,9 @@ public int getNumOpenConnections() } @Override - public Sequence run(final Query query, final Map context) + public Sequence run(final QueryPlus queryPlus, final Map context) { + final Query query = queryPlus.getQuery(); QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = QueryContexts.isBySegment(query); @@ -163,22 +200,38 @@ public Sequence run(final Query query, final Map context) final long requestStartTimeNs = System.nanoTime(); - final QueryMetrics> queryMetrics = toolChest.makeMetrics(query); - queryMetrics.server(host); + long timeoutAt = ((Long) context.get(QUERY_FAIL_TIME)).longValue(); + long maxScatterGatherBytes = QueryContexts.getMaxScatterGatherBytes(query); + AtomicLong totalBytesGathered = (AtomicLong) context.get(QUERY_TOTAL_BYTES_GATHERED); final HttpResponseHandler responseHandler = new HttpResponseHandler() { - private long responseStartTimeNs; private final AtomicLong byteCount = new AtomicLong(0); private final BlockingQueue queue = new LinkedBlockingQueue<>(); private final AtomicBoolean done = new AtomicBoolean(false); + private final AtomicReference fail = new AtomicReference<>(); + + private QueryMetrics> queryMetrics; + private long responseStartTimeNs; + + private QueryMetrics> acquireResponseMetrics() + { + if (queryMetrics == null) { + queryMetrics = toolChest.makeMetrics(query); + queryMetrics.server(host); + } + return queryMetrics; + } @Override public ClientResponse handleResponse(HttpResponse response) { + checkQueryTimeout(); + checkTotalBytesLimit(response.getContent().readableBytes()); + log.debug("Initial response from url[%s] for queryId[%s]", url, query.getId()); responseStartTimeNs = System.nanoTime(); - queryMetrics.reportNodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter); + acquireResponseMetrics().reportNodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter); try { final String responseContext = response.headers().get("X-Druid-Response-Context"); @@ -220,6 +273,11 @@ public int read() throws IOException @Override public boolean hasMoreElements() { + if (fail.get() != null) { + throw new RE(fail.get()); + } + checkQueryTimeout(); + // Done is always true until the last stream has be put in the queue. // Then the stream should be spouting good InputStreams. synchronized (done) { @@ -230,8 +288,17 @@ public boolean hasMoreElements() @Override public InputStream nextElement() { + if (fail.get() != null) { + throw new RE(fail.get()); + } + try { - return queue.take(); + InputStream is = queue.poll(checkQueryTimeout(), TimeUnit.MILLISECONDS); + if (is != null) { + return is; + } else { + throw new RE("Query[%s] url[%s] timed out.", query.getId(), url); + } } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -248,8 +315,13 @@ public ClientResponse handleChunk( ClientResponse clientResponse, HttpChunk chunk ) { + checkQueryTimeout(); + final ChannelBuffer channelBuffer = chunk.getContent(); final int bytes = channelBuffer.readableBytes(); + + checkTotalBytesLimit(bytes); + if (bytes > 0) { try { queue.put(new ChannelBufferInputStream(channelBuffer)); @@ -278,9 +350,10 @@ public ClientResponse done(ClientResponse clientRespon nodeTimeMs, byteCount.get() / (0.001 * nodeTimeMs) // Floating math; division by zero will yield Inf, not exception ); - queryMetrics.reportNodeTime(nodeTimeNs); - queryMetrics.reportNodeBytes(byteCount.get()); - queryMetrics.emit(emitter); + QueryMetrics> responseMetrics = acquireResponseMetrics(); + responseMetrics.reportNodeTime(nodeTimeNs); + responseMetrics.reportNodeBytes(byteCount.get()); + responseMetrics.emit(emitter); synchronized (done) { try { // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out @@ -306,34 +379,78 @@ public ClientResponse done(ClientResponse clientRespon @Override public void exceptionCaught(final ClientResponse clientResponse, final Throwable e) { - // Don't wait for lock in case the lock had something to do with the error - synchronized (done) { - done.set(true); - // Make a best effort to put a zero length buffer into the queue in case something is waiting on the take() - // If nothing is waiting on take(), this will be closed out anyways. - queue.offer( - new InputStream() - { - @Override - public int read() throws IOException - { - throw new IOException(e); - } - } + String msg = StringUtils.safeFormat( + "Query[%s] url[%s] failed with exception msg [%s]", + query.getId(), + url, + e.getMessage() + ); + setupResponseReadFailure(msg, e); + } + + private void setupResponseReadFailure(String msg, Throwable th) + { + fail.set(msg); + queue.clear(); + queue.offer(new InputStream() + { + @Override + public int read() throws IOException + { + if (th != null) { + throw new IOException(msg, th); + } else { + throw new IOException(msg); + } + } + }); + + } + + // Returns remaining timeout or throws exception if timeout already elapsed. + private long checkQueryTimeout() + { + long timeLeft = timeoutAt - System.currentTimeMillis(); + if (timeLeft <= 0) { + String msg = StringUtils.safeFormat("Query[%s] url[%s] timed out.", query.getId(), url); + setupResponseReadFailure(msg, null); + throw new RE(msg); + } else { + return timeLeft; + } + } + + private void checkTotalBytesLimit(long bytes) + { + if (maxScatterGatherBytes < Long.MAX_VALUE && totalBytesGathered.addAndGet(bytes) > maxScatterGatherBytes) { + String msg = StringUtils.safeFormat( + "Query[%s] url[%s] max scatter-gather bytes limit reached.", + query.getId(), + url ); + setupResponseReadFailure(msg, null); + throw new RE(msg); } } }; + + long timeLeft = timeoutAt - System.currentTimeMillis(); + + if (timeLeft <= 0) { + throw new RE("Query[%s] url[%s] timed out.", query.getId(), url); + } + future = httpClient.go( new Request( HttpMethod.POST, new URL(url) - ).setContent(objectMapper.writeValueAsBytes(query)) + ).setContent(objectMapper.writeValueAsBytes(QueryContexts.withTimeout(query, timeLeft))) .setHeader( HttpHeaders.Names.CONTENT_TYPE, isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON ), - responseHandler + responseHandler, + Duration.millis(timeLeft) ); queryWatcher.registerQuery(query, future); @@ -366,8 +483,10 @@ public void onFailure(Throwable t) ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON ), - new StatusResponseHandler(Charsets.UTF_8) - ).get(); + new StatusResponseHandler(Charsets.UTF_8), + Duration.standardSeconds(1) + ).get(1, TimeUnit.SECONDS); + if (res.getStatus().getCode() >= 500) { throw new RE( "Error cancelling query[%s]: queriable node returned status[%d] [%s].", @@ -376,7 +495,7 @@ public void onFailure(Throwable t) ); } } - catch (IOException | ExecutionException | InterruptedException e) { + catch (IOException | ExecutionException | InterruptedException | TimeoutException e) { Throwables.propagate(e); } } @@ -394,7 +513,7 @@ public void onFailure(Throwable t) @Override public JsonParserIterator make() { - return new JsonParserIterator(typeRef, future, url); + return new JsonParserIterator(typeRef, future, url, query); } @Override @@ -426,13 +545,15 @@ private class JsonParserIterator implements Iterator, Closeable private ObjectCodec objectCodec; private final JavaType typeRef; private final Future future; + private final Query query; private final String url; - public JsonParserIterator(JavaType typeRef, Future future, String url) + public JsonParserIterator(JavaType typeRef, Future future, String url, Query query) { this.typeRef = typeRef; this.future = future; this.url = url; + this.query = query; jp = null; } @@ -456,6 +577,7 @@ public boolean hasNext() public T next() { init(); + try { final T retVal = objectCodec.readValue(jp, typeRef); jp.nextToken(); @@ -476,7 +598,19 @@ private void init() { if (jp == null) { try { - jp = objectMapper.getFactory().createParser(future.get()); + InputStream is = future.get(); + if (is == null) { + throw new QueryInterruptedException( + new ResourceLimitExceededException( + "query[%s] url[%s] timed out or max bytes limit reached.", + query.getId(), + url + ), + host + ); + } else { + jp = objectMapper.getFactory().createParser(is); + } final JsonToken nextToken = jp.nextToken(); if (nextToken == JsonToken.START_OBJECT) { QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class); @@ -489,7 +623,13 @@ private void init() } } catch (IOException | InterruptedException | ExecutionException e) { - throw new RE(e, "Failure getting results from[%s] because of [%s]", url, e.getMessage()); + throw new RE( + e, + "Failure getting results for query[%s] url[%s] because of [%s]", + query.getId(), + url, + e.getMessage() + ); } catch (CancellationException e) { throw new QueryInterruptedException(e, host); diff --git a/server/src/main/java/io/druid/client/DruidServer.java b/server/src/main/java/io/druid/client/DruidServer.java index 1a05b29422cc..0f0c4ff6999a 100644 --- a/server/src/main/java/io/druid/client/DruidServer.java +++ b/server/src/main/java/io/druid/client/DruidServer.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import java.util.Collections; @@ -57,7 +58,7 @@ public class DruidServer implements Comparable public DruidServer( DruidNode node, DruidServerConfig config, - String type + ServerType type ) { this( @@ -75,7 +76,7 @@ public DruidServer( @JsonProperty("name") String name, @JsonProperty("host") String host, @JsonProperty("maxSize") long maxSize, - @JsonProperty("type") String type, + @JsonProperty("type") ServerType type, @JsonProperty("tier") String tier, @JsonProperty("priority") int priority ) @@ -86,6 +87,7 @@ public DruidServer( this.segments = new ConcurrentHashMap(); } + @JsonProperty public String getName() { return metadata.getName(); @@ -96,6 +98,7 @@ public DruidServerMetadata getMetadata() return metadata; } + @JsonProperty public String getHost() { return metadata.getHost(); @@ -106,26 +109,30 @@ public long getCurrSize() return currSize; } + @JsonProperty public long getMaxSize() { return metadata.getMaxSize(); } - public String getType() + @JsonProperty + public ServerType getType() { return metadata.getType(); } + @JsonProperty public String getTier() { return metadata.getTier(); } - public boolean isAssignable() + public boolean segmentReplicatable() { - return metadata.isAssignable(); + return metadata.segmentReplicatable(); } + @JsonProperty public int getPriority() { return metadata.getPriority(); @@ -226,6 +233,15 @@ public Iterable getDataSources() return dataSources.values(); } + public void removeAllSegments() + { + synchronized (lock) { + dataSources.clear(); + segments.clear(); + currSize = 0; + } + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/io/druid/client/DruidServerDiscovery.java b/server/src/main/java/io/druid/client/DruidServerDiscovery.java new file mode 100644 index 000000000000..7b28ef6850ae --- /dev/null +++ b/server/src/main/java/io/druid/client/DruidServerDiscovery.java @@ -0,0 +1,174 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.metamx.emitter.EmittingLogger; +import io.druid.concurrent.Execs; +import io.druid.curator.inventory.CuratorInventoryManager; +import io.druid.curator.inventory.CuratorInventoryManagerStrategy; +import io.druid.curator.inventory.InventoryManagerConfig; +import io.druid.java.util.common.ISE; +import org.apache.curator.framework.CuratorFramework; + +import java.io.IOException; + +/** + * Discovers DruidServer instances that serve segments using CuratorInventoryManager. + */ +public class DruidServerDiscovery +{ + private final EmittingLogger log = new EmittingLogger(DruidServerDiscovery.class); + private final CuratorInventoryManager curatorInventoryManager; + private volatile Listener listener; + + DruidServerDiscovery( + final CuratorFramework curatorFramework, + final String announcementsPath, + final ObjectMapper jsonMapper + ) + { + curatorInventoryManager = initCuratorInventoryManager(curatorFramework, announcementsPath, jsonMapper); + } + + public void start() throws Exception + { + Preconditions.checkNotNull(listener, "listener is not configured yet"); + curatorInventoryManager.start(); + } + + public void stop() throws IOException + { + curatorInventoryManager.stop(); + } + + private CuratorInventoryManager initCuratorInventoryManager( + final CuratorFramework curator, + final String announcementsPath, + final ObjectMapper jsonMapper + ) + { + return new CuratorInventoryManager<>( + curator, + new InventoryManagerConfig() + { + @Override + public String getContainerPath() + { + return announcementsPath; + } + + @Override + public String getInventoryPath() + { + return "/NON_EXISTENT_DUMMY_INVENTORY_PATH"; + } + }, + Execs.singleThreaded("CuratorInventoryManagerBasedServerDiscovery-%s"), + new CuratorInventoryManagerStrategy() + { + @Override + public DruidServer deserializeContainer(byte[] bytes) + { + try { + return jsonMapper.readValue(bytes, DruidServer.class); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + @Override + public void newContainer(DruidServer container) + { + log.info("New Server[%s]", container.getName()); + listener.serverAdded(container); + } + + @Override + public void deadContainer(DruidServer container) + { + log.info("Server Disappeared[%s]", container.getName()); + listener.serverRemoved(container); + } + + @Override + public DruidServer updateContainer(DruidServer oldContainer, DruidServer newContainer) + { + log.info("Server updated[%s]", oldContainer.getName()); + return listener.serverUpdated(oldContainer, newContainer); + } + + @Override + public Object deserializeInventory(byte[] bytes) + { + throw new ISE("no inventory should exist."); + } + + @Override + public DruidServer addInventory( + final DruidServer container, + String inventoryKey, + final Object inventory + ) + { + throw new ISE("no inventory should exist."); + } + + @Override + public DruidServer updateInventory( + DruidServer container, String inventoryKey, Object inventory + ) + { + throw new ISE("no inventory should exist."); + } + + @Override + public DruidServer removeInventory(final DruidServer container, String inventoryKey) + { + throw new ISE("no inventory should exist."); + } + + @Override + public void inventoryInitialized() + { + log.info("Server inventory initialized."); + listener.initialized(); + } + } + ); + } + + public void registerListener(Listener listener) + { + Preconditions.checkArgument(this.listener == null, "listener registered already."); + this.listener = listener; + } + + public interface Listener + { + void serverAdded(DruidServer server); + DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer); + void serverRemoved(DruidServer server); + void initialized(); + } +} diff --git a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java new file mode 100644 index 000000000000..6dbb7cf4bcb6 --- /dev/null +++ b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java @@ -0,0 +1,78 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Predicates; +import com.metamx.http.client.HttpClient; +import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Smile; +import io.druid.java.util.common.Pair; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.ZkPathsConfig; +import io.druid.timeline.DataSegment; +import org.apache.curator.framework.CuratorFramework; + +import javax.validation.constraints.NotNull; + +/** + */ +public class FilteredHttpServerInventoryViewProvider implements FilteredServerInventoryViewProvider +{ + @JacksonInject + @NotNull + @Client + HttpClient httpClient = null; + + @JacksonInject + @NotNull + @Smile + ObjectMapper smileMapper = null; + + @JacksonInject + @NotNull + @Json + ObjectMapper jsonMapper = null; + + @JacksonInject + @NotNull + HttpServerInventoryViewConfig config = null; + + @JacksonInject + @NotNull + private ZkPathsConfig zkPaths = null; + + @JacksonInject + @NotNull + private CuratorFramework curator = null; + + @Override + public HttpServerInventoryView get() + { + return new HttpServerInventoryView( + jsonMapper, smileMapper, httpClient, + new DruidServerDiscovery(curator, zkPaths.getAnnouncementsPath(), jsonMapper), + Predicates.>alwaysTrue(), + config + ); + } +} diff --git a/server/src/main/java/io/druid/client/FilteredServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/FilteredServerInventoryViewProvider.java index f6f7fa746085..c64cb4aa3c6d 100644 --- a/server/src/main/java/io/druid/client/FilteredServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/FilteredServerInventoryViewProvider.java @@ -27,7 +27,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = FilteredBatchServerInventoryViewProvider.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "legacy", value = FilteredSingleServerInventoryViewProvider.class), - @JsonSubTypes.Type(name = "batch", value = FilteredBatchServerInventoryViewProvider.class) + @JsonSubTypes.Type(name = "batch", value = FilteredBatchServerInventoryViewProvider.class), + @JsonSubTypes.Type(name = "http", value = FilteredHttpServerInventoryViewProvider.class) }) public interface FilteredServerInventoryViewProvider extends Provider { diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryView.java b/server/src/main/java/io/druid/client/HttpServerInventoryView.java new file mode 100644 index 000000000000..c2bb8eee61b8 --- /dev/null +++ b/server/src/main/java/io/druid/client/HttpServerInventoryView.java @@ -0,0 +1,663 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.MapMaker; +import com.google.common.net.HostAndPort; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.inject.Inject; +import com.metamx.emitter.EmittingLogger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.Request; +import com.metamx.http.client.io.AppendableByteArrayInputStream; +import com.metamx.http.client.response.ClientResponse; +import com.metamx.http.client.response.InputStreamResponseHandler; +import io.druid.concurrent.LifecycleLock; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Smile; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.server.coordination.DataSegmentChangeCallback; +import io.druid.server.coordination.DataSegmentChangeHandler; +import io.druid.server.coordination.DataSegmentChangeRequest; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.SegmentChangeRequestHistory; +import io.druid.server.coordination.SegmentChangeRequestsSnapshot; +import io.druid.timeline.DataSegment; +import org.jboss.netty.handler.codec.http.HttpHeaders; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.joda.time.Duration; + +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; + +/** + * This class uses CuratorInventoryManager to listen for queryable server membership which serve segments(e.g. Historicals). + * For each queryable server, it uses HTTP GET /druid-internal/v1/segments (see docs in SegmentListerResource.getSegments(..). + */ +public class HttpServerInventoryView implements ServerInventoryView, FilteredServerInventoryView +{ + private final EmittingLogger log = new EmittingLogger(HttpServerInventoryView.class); + private final DruidServerDiscovery serverDiscovery; + + private final LifecycleLock lifecycleLock = new LifecycleLock(); + + private final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); + private final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); + + private final ConcurrentMap>> segmentPredicates = new MapMaker() + .makeMap(); + private final Predicate> defaultFilter; + private volatile Predicate> finalPredicate; + + // For each queryable server, a name -> DruidServerHolder entry is kept + private final Map servers = new HashMap<>(); + + private volatile ExecutorService executor; + + // a queue of queryable server names for which worker threads in executor initiate the segment list call i.e. + // DruidServerHolder.updateSegmentsListAsync(..) which updates the segment list asynchronously and adds itself + // to this queue again for next update. + private final BlockingQueue queue = new LinkedBlockingDeque<>(); + + + + private final HttpClient httpClient; + private final ObjectMapper smileMapper; + private final HttpServerInventoryViewConfig config; + + @Inject + public HttpServerInventoryView( + final @Json ObjectMapper jsonMapper, + final @Smile ObjectMapper smileMapper, + final @Global HttpClient httpClient, + final DruidServerDiscovery serverDiscovery, + final Predicate> defaultFilter, + final HttpServerInventoryViewConfig config + ) + { + this.httpClient = httpClient; + this.smileMapper = smileMapper; + this.serverDiscovery = serverDiscovery; + this.defaultFilter = defaultFilter; + this.finalPredicate = defaultFilter; + this.config = config; + } + + + @LifecycleStart + public void start() throws Exception + { + synchronized (lifecycleLock) { + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + + log.info("Starting HttpServerInventoryView."); + + try { + executor = Executors.newFixedThreadPool( + config.getNumThreads(), + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("HttpServerInventoryView-%s").build() + ); + + executor.execute( + new Runnable() + { + @Override + public void run() + { + if (!lifecycleLock.awaitStarted()) { + log.error("WTF! lifecycle not started, segments will not be discovered."); + return; + } + + while (!Thread.interrupted() && lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) { + try { + String name = queue.take(); + + synchronized (servers) { + DruidServerHolder holder = servers.get(name); + if (holder != null) { + holder.updateSegmentsListAsync(); + } + } + } + catch (InterruptedException ex) { + log.info("main thread interrupted, served segments list is not synced anymore."); + Thread.currentThread().interrupt(); + } + catch (Throwable th) { + log.makeAlert(th, "main thread ignored error").emit(); + } + } + + log.info("HttpServerInventoryView main thread exited."); + } + } + ); + + serverDiscovery.registerListener( + new DruidServerDiscovery.Listener() + { + @Override + public void serverAdded(DruidServer server) + { + serverAddedOrUpdated(server); + } + + @Override + public DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer) + { + return serverAddedOrUpdated(newServer); + } + + @Override + public void serverRemoved(DruidServer server) + { + HttpServerInventoryView.this.serverRemoved(server); + runServerCallbacks(server); + } + + @Override + public void initialized() + { + serverInventoryInitialized(); + } + } + ); + serverDiscovery.start(); + + log.info("Started HttpServerInventoryView."); + lifecycleLock.started(); + } finally { + lifecycleLock.exitStart(); + } + } + } + + @LifecycleStop + public void stop() throws IOException + { + synchronized (lifecycleLock) { + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); + } + + log.info("Stopping HttpServerInventoryView."); + + serverDiscovery.stop(); + + if (executor != null) { + executor.shutdownNow(); + executor = null; + } + + queue.clear(); + + log.info("Stopped HttpServerInventoryView."); + } + } + + @Override + public void registerSegmentCallback( + Executor exec, SegmentCallback callback, Predicate> filter + ) + { + segmentCallbacks.put(callback, exec); + segmentPredicates.put(callback, filter); + + finalPredicate = Predicates.or( + defaultFilter, + Predicates.or(segmentPredicates.values()) + ); + } + + @Override + public void registerServerCallback(Executor exec, ServerCallback callback) + { + serverCallbacks.put(callback, exec); + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + segmentCallbacks.put(callback, exec); + } + + @Override + public DruidServer getInventoryValue(String containerKey) + { + synchronized (servers) { + DruidServerHolder holder = servers.get(containerKey); + if (holder != null) { + return holder.druidServer; + } + } + + return null; + } + + @Override + public Iterable getInventory() + { + synchronized (servers) { + return Iterables.transform( + servers.values(), new com.google.common.base.Function() + { + @Override + public DruidServer apply(DruidServerHolder input) + { + return input.druidServer; + } + } + ); + } + } + + private void runSegmentCallbacks( + final Function fn + ) + { + for (final Map.Entry entry : segmentCallbacks.entrySet()) { + entry.getValue().execute( + new Runnable() + { + @Override + public void run() + { + if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { + segmentCallbacks.remove(entry.getKey()); + if (segmentPredicates.remove(entry.getKey()) != null) { + finalPredicate = Predicates.or( + defaultFilter, + Predicates.or(segmentPredicates.values()) + ); + } + } + } + } + ); + } + } + + private void runServerCallbacks(final DruidServer server) + { + for (final Map.Entry entry : serverCallbacks.entrySet()) { + entry.getValue().execute( + new Runnable() + { + @Override + public void run() + { + if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) { + serverCallbacks.remove(entry.getKey()); + } + } + } + ); + } + } + + //best effort wait for first segment listing fetch from all servers and then call + //segmentViewInitialized on all registered segment callbacks. + private void serverInventoryInitialized() + { + for (DruidServerHolder server : servers.values()) { + server.awaitInitialization(); + } + + log.info("Calling SegmentCallback.segmentViewInitialized() for all callbacks."); + + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentViewInitialized(); + } + } + ); + } + + private DruidServer serverAddedOrUpdated(DruidServer server) + { + DruidServerHolder curr; + DruidServerHolder newHolder; + synchronized (servers) { + curr = servers.get(server.getName()); + newHolder = curr == null ? new DruidServerHolder(server) : curr.updatedHolder(server); + servers.put(server.getName(), newHolder); + } + + newHolder.updateSegmentsListAsync(); + + return newHolder.druidServer; + } + + private void serverRemoved(DruidServer server) + { + synchronized (servers) { + servers.remove(server.getName()); + } + } + + public DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer) + { + return serverAddedOrUpdated(newServer); + } + + @Override + public boolean isStarted() + { + return lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS); + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + synchronized (servers) { + DruidServerHolder holder = servers.get(serverKey); + if (holder != null) { + return holder.druidServer.getSegment(segment.getIdentifier()) != null; + } else { + return false; + } + } + } + + private class DruidServerHolder + { + private final Object lock = new Object(); + + //lock is used to keep state in counter and and segment list in druidServer consistent + // so that in "updateHolder()" method, new DruidServerHolder with updated DruidServer info + // can be safely created + private final DruidServer druidServer; + + private volatile SegmentChangeRequestHistory.Counter counter = null; + + private final HostAndPort serverHostAndPort; + + private final DataSegmentChangeHandler changeHandler; + private final long serverHttpTimeout = config.getServerTimeout() + 1000; + + private final CountDownLatch initializationLatch = new CountDownLatch(1); + + DruidServerHolder(DruidServer druidServer) + { + this(druidServer, null); + } + + private DruidServerHolder(final DruidServer druidServer, final SegmentChangeRequestHistory.Counter counter) + { + this.druidServer = druidServer; + this.serverHostAndPort = HostAndPort.fromString(druidServer.getHost()); + this.counter = counter; + changeHandler = new DataSegmentChangeHandler() + { + @Override + public void addSegment( + final DataSegment segment, final DataSegmentChangeCallback callback + ) + { + if (finalPredicate.apply(Pair.of(druidServer.getMetadata(), segment))) { + druidServer.addDataSegment(segment.getIdentifier(), segment); + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentAdded(druidServer.getMetadata(), segment); + } + } + ); + } + } + + @Override + public void removeSegment( + final DataSegment segment, final DataSegmentChangeCallback callback + ) + { + druidServer.removeDataSegment(segment.getIdentifier()); + + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentRemoved(druidServer.getMetadata(), segment); + } + } + ); + } + }; + } + + //wait for first fetch of segment listing from server. + void awaitInitialization() + { + try { + if (!initializationLatch.await(serverHttpTimeout, TimeUnit.MILLISECONDS)) { + log.warn("Await initialization timed out for server [%s].", druidServer.getName()); + } + } catch (InterruptedException ex) { + log.warn("Await initialization interrupted while waiting on server [%s].", druidServer.getName()); + Thread.currentThread().interrupt(); + } + } + + DruidServerHolder updatedHolder(DruidServer server) + { + synchronized (lock) { + return new DruidServerHolder(server.addDataSegments(druidServer), counter) ; + } + } + + Future updateSegmentsListAsync() + { + try { + final String req; + if (counter != null) { + req = String.format( + "/druid-internal/v1/segments?counter=%s&hash=%s&timeout=%s", + counter.getCounter(), + counter.getHash(), + config.getServerTimeout() + ); + } else { + req = String.format( + "/druid-internal/v1/segments?counter=-1&timeout=%s", + config.getServerTimeout() + ); + } + URL url = new URL("http", serverHostAndPort.getHostText(), serverHostAndPort.getPort(), req); + + BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); + + log.debug("Sending segment list fetch request to [%s] on URL [%s]", druidServer.getName(), url); + + ListenableFuture future = httpClient.go( + new Request(HttpMethod.GET, url) + .addHeader( + HttpHeaders.Names.ACCEPT, + SmileMediaTypes.APPLICATION_JACKSON_SMILE + ) + .addHeader(HttpHeaders.Names.CONTENT_TYPE, SmileMediaTypes.APPLICATION_JACKSON_SMILE), + responseHandler, + new Duration(serverHttpTimeout) + ); + + log.debug("Sent segment list fetch request to [%s]", druidServer.getName()); + + Futures.addCallback( + future, + new FutureCallback() + { + @Override + public void onSuccess(InputStream stream) + { + try { + if (responseHandler.status == HttpServletResponse.SC_NO_CONTENT) { + log.debug("Received NO CONTENT from [%s]", druidServer.getName()); + return; + } else if (responseHandler.status != HttpServletResponse.SC_OK) { + onFailure(null); + return; + } + + log.debug("Received segment list response from [%s]", druidServer.getName()); + + SegmentChangeRequestsSnapshot delta = smileMapper.readValue( + stream, + SegmentChangeRequestsSnapshot.class + ); + + log.debug("Finished reading segment list response from [%s]", druidServer.getName()); + + synchronized (lock) { + if (delta.isResetCounter()) { + log.debug( + "Server [%s] requested resetCounter for reason [%s].", + druidServer.getName(), + delta.getResetCause() + ); + counter = null; + return; + } + + if (counter == null) { + druidServer.removeAllSegments(); + } + + for (DataSegmentChangeRequest request : delta.getRequests()) { + request.go(changeHandler, null); + } + counter = delta.getCounter(); + } + + initializationLatch.countDown(); + } + catch (Exception ex) { + log.error(ex, "error processing segment list response from server [%s]", druidServer.getName()); + } + finally { + queue.add(druidServer.getName()); + } + } + + @Override + public void onFailure(Throwable t) + { + try { + if (t != null) { + log.error( + t, + "failed to fetch segment list from server [%s]. Return code [%s], Reason: [%s]", + druidServer.getName(), + responseHandler.status, + responseHandler.description + ); + } else { + log.error( + "failed to fetch segment list from server [%s]. Return code [%s], Reason: [%s]", + druidServer.getName(), + responseHandler.status, + responseHandler.description + ); + } + + // sleep for a bit so that retry does not happen immediately. + try { + Thread.sleep(5000); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + finally { + queue.add(druidServer.getName()); + } + } + }, + executor + ); + + return future; + } catch (Throwable th) { + queue.add(druidServer.getName()); + log.makeAlert(th, "Fatal error while fetching segment list from server [%s].", druidServer.getName()).emit(); + + // sleep for a bit so that retry does not happen immediately. + try { + Thread.sleep(5000); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + + throw Throwables.propagate(th); + } + } + } + + private static class BytesAccumulatingResponseHandler extends InputStreamResponseHandler + { + private int status; + private String description; + + @Override + public ClientResponse handleResponse(HttpResponse response) + { + status = response.getStatus().getCode(); + description = response.getStatus().getReasonPhrase(); + return ClientResponse.unfinished(super.handleResponse(response).getObj()); + } + } +} diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryViewConfig.java b/server/src/main/java/io/druid/client/HttpServerInventoryViewConfig.java new file mode 100644 index 000000000000..6375b875ec01 --- /dev/null +++ b/server/src/main/java/io/druid/client/HttpServerInventoryViewConfig.java @@ -0,0 +1,61 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.joda.time.Period; + +/** + */ +public class HttpServerInventoryViewConfig +{ + @JsonProperty + private final long serverTimeout; + + @JsonProperty + private final int numThreads; + + @JsonCreator + public HttpServerInventoryViewConfig( + @JsonProperty("serverTimeout") Period serverTimeout, + @JsonProperty("numThreads") Integer numThreads + ){ + this.serverTimeout = serverTimeout != null + ? serverTimeout.toStandardDuration().getMillis() + : 4*60*1000; //4 mins + + this.numThreads = numThreads != null ? numThreads.intValue() : 5; + + Preconditions.checkArgument(this.serverTimeout > 0, "server timeout must be > 0 ms"); + Preconditions.checkArgument(this.numThreads > 1, "numThreads must be > 1"); + } + + public long getServerTimeout() + { + return serverTimeout; + } + + public int getNumThreads() + { + return numThreads; + } +} diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java new file mode 100644 index 000000000000..efc109542ec2 --- /dev/null +++ b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java @@ -0,0 +1,80 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Predicates; +import com.metamx.http.client.HttpClient; +import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Smile; +import io.druid.java.util.common.Pair; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.ZkPathsConfig; +import io.druid.timeline.DataSegment; +import org.apache.curator.framework.CuratorFramework; + +import javax.validation.constraints.NotNull; + +/** + */ +public class HttpServerInventoryViewProvider implements ServerInventoryViewProvider +{ + @JacksonInject + @NotNull + @Client + HttpClient httpClient = null; + + @JacksonInject + @NotNull + @Smile + ObjectMapper smileMapper = null; + + @JacksonInject + @NotNull + @Json + ObjectMapper jsonMapper = null; + + @JacksonInject + @NotNull + HttpServerInventoryViewConfig config = null; + + @JacksonInject + @NotNull + private ZkPathsConfig zkPaths = null; + + @JacksonInject + @NotNull + private CuratorFramework curator = null; + + @Override + public HttpServerInventoryView get() + { + return new HttpServerInventoryView( + jsonMapper, + smileMapper, + httpClient, + new DruidServerDiscovery(curator, zkPaths.getAnnouncementsPath(), jsonMapper), + Predicates.>alwaysTrue(), + config + ); + } +} diff --git a/server/src/main/java/io/druid/client/ImmutableDruidDataSource.java b/server/src/main/java/io/druid/client/ImmutableDruidDataSource.java index 1c7435e0e755..c83d674e4285 100644 --- a/server/src/main/java/io/druid/client/ImmutableDruidDataSource.java +++ b/server/src/main/java/io/druid/client/ImmutableDruidDataSource.java @@ -72,4 +72,15 @@ public Set getSegments() { return segmentsHolder; } + + @Override + public String toString() + { + // partitionNames is intentionally ignored because it is usually large + return "ImmutableDruidDataSource{" + + "name='" + name + + "', segments='" + segmentsHolder + + "', properties='" + properties + + "'}"; + } } diff --git a/server/src/main/java/io/druid/client/ImmutableDruidServer.java b/server/src/main/java/io/druid/client/ImmutableDruidServer.java index 534f91cad445..1851befe277e 100644 --- a/server/src/main/java/io/druid/client/ImmutableDruidServer.java +++ b/server/src/main/java/io/druid/client/ImmutableDruidServer.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import java.util.Map; @@ -72,7 +73,7 @@ public long getMaxSize() return metadata.getMaxSize(); } - public String getType() + public ServerType getType() { return metadata.getType(); } @@ -106,4 +107,15 @@ public Map getSegments() { return segments; } + + @Override + public String toString() + { + // segments is intentionally ignored because it is usually large + return "ImmutableDruidServer{" + + "meta='" + metadata + + "', size='" + currSize + + "', sources='" + dataSources + + "'}"; + } } diff --git a/server/src/main/java/io/druid/client/InventoryView.java b/server/src/main/java/io/druid/client/InventoryView.java index 4956f1f65c6c..fd9ea0c0b2b0 100644 --- a/server/src/main/java/io/druid/client/InventoryView.java +++ b/server/src/main/java/io/druid/client/InventoryView.java @@ -19,10 +19,14 @@ package io.druid.client; +import io.druid.timeline.DataSegment; + /** */ public interface InventoryView { - public DruidServer getInventoryValue(String string); - public Iterable getInventory(); + DruidServer getInventoryValue(String string); + Iterable getInventory(); + boolean isStarted(); + boolean isSegmentLoadedByServer(String serverKey, DataSegment segment); } diff --git a/server/src/main/java/io/druid/client/ServerInventoryView.java b/server/src/main/java/io/druid/client/ServerInventoryView.java index 8cc61f9a37cf..27ba7e7d4f5f 100644 --- a/server/src/main/java/io/druid/client/ServerInventoryView.java +++ b/server/src/main/java/io/druid/client/ServerInventoryView.java @@ -19,329 +19,9 @@ package io.druid.client; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; -import com.google.common.base.Function; -import com.google.common.base.Throwables; -import com.google.common.collect.MapMaker; -import com.metamx.emitter.EmittingLogger; -import io.druid.concurrent.Execs; -import io.druid.curator.inventory.CuratorInventoryManager; -import io.druid.curator.inventory.CuratorInventoryManagerStrategy; -import io.druid.curator.inventory.InventoryManagerConfig; -import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.lifecycle.LifecycleStart; -import io.druid.java.util.common.lifecycle.LifecycleStop; -import io.druid.timeline.DataSegment; -import org.apache.curator.framework.CuratorFramework; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.CharBuffer; -import java.util.Map; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicBoolean; - /** + * Marker interface for making batch/single/http server inventory view configurable. */ -public abstract class ServerInventoryView implements ServerView, InventoryView +public interface ServerInventoryView extends ServerView, InventoryView { - - private final EmittingLogger log; - private final CuratorInventoryManager inventoryManager; - private final AtomicBoolean started = new AtomicBoolean(false); - - private final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); - private final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); - - public ServerInventoryView( - final EmittingLogger log, - final String announcementsPath, - final String inventoryPath, - final CuratorFramework curator, - final ObjectMapper jsonMapper, - final TypeReference typeReference - ) - { - this.log = log; - this.inventoryManager = new CuratorInventoryManager<>( - curator, - new InventoryManagerConfig() - { - @Override - public String getContainerPath() - { - return announcementsPath; - } - - @Override - public String getInventoryPath() - { - return inventoryPath; - } - }, - Execs.singleThreaded("ServerInventoryView-%s"), - new CuratorInventoryManagerStrategy() - { - @Override - public DruidServer deserializeContainer(byte[] bytes) - { - try { - return jsonMapper.readValue(bytes, DruidServer.class); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - @Override - public InventoryType deserializeInventory(byte[] bytes) - { - try { - return jsonMapper.readValue(bytes, typeReference); - } - catch (IOException e) { - CharBuffer.wrap(StringUtils.fromUtf8(bytes).toCharArray()); - CharBuffer charBuffer = Charsets.UTF_8.decode(ByteBuffer.wrap(bytes)); - log.error(e, "Could not parse json: %s", charBuffer.toString()); - throw Throwables.propagate(e); - } - } - - @Override - public void newContainer(DruidServer container) - { - log.info("New Server[%s]", container); - } - - @Override - public void deadContainer(DruidServer deadContainer) - { - log.info("Server Disappeared[%s]", deadContainer); - runServerCallbacks(deadContainer); - } - - @Override - public DruidServer updateContainer(DruidServer oldContainer, DruidServer newContainer) - { - return newContainer.addDataSegments(oldContainer); - } - - @Override - public DruidServer addInventory( - final DruidServer container, - String inventoryKey, - final InventoryType inventory - ) - { - return addInnerInventory(container, inventoryKey, inventory); - } - - @Override - public DruidServer updateInventory( - DruidServer container, String inventoryKey, InventoryType inventory - ) - { - return updateInnerInventory(container, inventoryKey, inventory); - } - - @Override - public DruidServer removeInventory(final DruidServer container, String inventoryKey) - { - return removeInnerInventory(container, inventoryKey); - } - - @Override - public void inventoryInitialized() - { - log.info("Inventory Initialized"); - runSegmentCallbacks( - new Function() - { - @Override - public CallbackAction apply(SegmentCallback input) - { - return input.segmentViewInitialized(); - } - } - ); - } - } - ); - } - - @LifecycleStart - public void start() throws Exception - { - synchronized (started) { - if (!started.get()) { - inventoryManager.start(); - started.set(true); - } - } - } - - @LifecycleStop - public void stop() throws IOException - { - synchronized (started) { - if (started.getAndSet(false)) { - inventoryManager.stop(); - } - } - } - - public boolean isStarted() - { - return started.get(); - } - - @Override - public DruidServer getInventoryValue(String containerKey) - { - return inventoryManager.getInventoryValue(containerKey); - } - - @Override - public Iterable getInventory() - { - return inventoryManager.getInventory(); - } - - @Override - public void registerServerCallback(Executor exec, ServerCallback callback) - { - serverCallbacks.put(callback, exec); - } - - @Override - public void registerSegmentCallback(Executor exec, SegmentCallback callback) - { - segmentCallbacks.put(callback, exec); - } - - public InventoryManagerConfig getInventoryManagerConfig() - { - return inventoryManager.getConfig(); - } - - protected void runSegmentCallbacks( - final Function fn - ) - { - for (final Map.Entry entry : segmentCallbacks.entrySet()) { - entry.getValue().execute( - new Runnable() - { - @Override - public void run() - { - if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { - segmentCallbackRemoved(entry.getKey()); - segmentCallbacks.remove(entry.getKey()); - } - } - } - ); - } - } - - protected void runServerCallbacks(final DruidServer server) - { - for (final Map.Entry entry : serverCallbacks.entrySet()) { - entry.getValue().execute( - new Runnable() - { - @Override - public void run() - { - if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) { - serverCallbacks.remove(entry.getKey()); - } - } - } - ); - } - } - - protected void addSingleInventory( - final DruidServer container, - final DataSegment inventory - ) - { - log.debug("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); - - if (container.getSegment(inventory.getIdentifier()) != null) { - log.warn( - "Not adding or running callbacks for existing segment[%s] on server[%s]", - inventory.getIdentifier(), - container.getName() - ); - - return; - } - - container.addDataSegment(inventory.getIdentifier(), inventory); - - runSegmentCallbacks( - new Function() - { - @Override - public CallbackAction apply(SegmentCallback input) - { - return input.segmentAdded(container.getMetadata(), inventory); - } - } - ); - } - - protected void removeSingleInventory(final DruidServer container, String inventoryKey) - { - log.debug("Server[%s] removed segment[%s]", container.getName(), inventoryKey); - final DataSegment segment = container.getSegment(inventoryKey); - - if (segment == null) { - log.warn( - "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]", - inventoryKey, - container.getName() - ); - - return; - } - - container.removeDataSegment(inventoryKey); - - runSegmentCallbacks( - new Function() - { - @Override - public CallbackAction apply(SegmentCallback input) - { - return input.segmentRemoved(container.getMetadata(), segment); - } - } - ); - } - - protected abstract DruidServer addInnerInventory( - final DruidServer container, - String inventoryKey, - final InventoryType inventory - ); - - protected abstract DruidServer updateInnerInventory( - final DruidServer container, - String inventoryKey, - final InventoryType inventory - ); - - protected abstract DruidServer removeInnerInventory( - final DruidServer container, - String inventoryKey - ); - - protected abstract void segmentCallbackRemoved(SegmentCallback callback); } diff --git a/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java index 6a9c05e7288a..2a0f2f17ac96 100644 --- a/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java @@ -28,7 +28,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = BatchServerInventoryViewProvider.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "legacy", value = SingleServerInventoryProvider.class), - @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class) + @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class), + @JsonSubTypes.Type(name = "http", value = HttpServerInventoryViewProvider.class), }) public interface ServerInventoryViewProvider extends Provider { diff --git a/server/src/main/java/io/druid/client/SingleServerInventoryView.java b/server/src/main/java/io/druid/client/SingleServerInventoryView.java index 298050f87d89..f54d6a10ecb3 100644 --- a/server/src/main/java/io/druid/client/SingleServerInventoryView.java +++ b/server/src/main/java/io/druid/client/SingleServerInventoryView.java @@ -40,7 +40,7 @@ /** */ @ManageLifecycle -public class SingleServerInventoryView extends ServerInventoryView implements FilteredServerInventoryView +public class SingleServerInventoryView extends AbstractCuratorServerInventoryView implements FilteredServerInventoryView { private static final EmittingLogger log = new EmittingLogger(SingleServerInventoryView.class); @@ -101,6 +101,7 @@ protected DruidServer removeInnerInventory(DruidServer container, String invento return container; } + @Override public void registerSegmentCallback( final Executor exec, final SegmentCallback callback, diff --git a/server/src/main/java/io/druid/client/cache/MapCache.java b/server/src/main/java/io/druid/client/cache/MapCache.java index 7dfac4c233de..fe058523e5af 100644 --- a/server/src/main/java/io/druid/client/cache/MapCache.java +++ b/server/src/main/java/io/druid/client/cache/MapCache.java @@ -165,6 +165,7 @@ private ByteBuffer computeKey(byte[] idBytes, byte[] key) return retVal; } + @Override public boolean isLocal() { return true; diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCache.java b/server/src/main/java/io/druid/client/cache/MemcachedCache.java index ab63dce819db..91357c589712 100644 --- a/server/src/main/java/io/druid/client/cache/MemcachedCache.java +++ b/server/src/main/java/io/druid/client/cache/MemcachedCache.java @@ -409,7 +409,7 @@ public MemcachedClientIF get() { Preconditions.checkArgument( config.getMemcachedPrefix().length() <= MAX_PREFIX_LENGTH, - "memcachedPrefix length [%d] exceeds maximum length [%d]", + "memcachedPrefix length [%s] exceeds maximum length [%s]", config.getMemcachedPrefix().length(), MAX_PREFIX_LENGTH ); @@ -603,6 +603,7 @@ private static String computeKeyHash(String memcachedPrefix, NamedKey key) return memcachedPrefix + ":" + DigestUtils.sha1Hex(key.namespace) + ":" + DigestUtils.sha1Hex(key.key); } + @Override public boolean isLocal() { return false; diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCustomConnectionFactoryBuilder.java b/server/src/main/java/io/druid/client/cache/MemcachedCustomConnectionFactoryBuilder.java index 4cb4bf65469d..ff765e7bfd7a 100644 --- a/server/src/main/java/io/druid/client/cache/MemcachedCustomConnectionFactoryBuilder.java +++ b/server/src/main/java/io/druid/client/cache/MemcachedCustomConnectionFactoryBuilder.java @@ -113,6 +113,7 @@ public HashAlgorithm getHashAlg() { return hashAlg == null ? super.getHashAlg() : hashAlg; } + @Override public Collection getInitialObservers() { return initialObservers; } diff --git a/server/src/main/java/io/druid/client/selector/AbstractTierSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/AbstractTierSelectorStrategy.java index 8de7dfa6ecca..60fcac019dc2 100644 --- a/server/src/main/java/io/druid/client/selector/AbstractTierSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/AbstractTierSelectorStrategy.java @@ -19,12 +19,13 @@ package io.druid.client.selector; -import io.druid.java.util.common.ISE; +import com.google.common.collect.Iterables; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; -import java.util.Map; +import java.util.ArrayList; +import java.util.List; import java.util.Set; -import java.util.TreeMap; /** */ @@ -39,23 +40,27 @@ public AbstractTierSelectorStrategy(ServerSelectorStrategy serverSelectorStrateg @Override public QueryableDruidServer pick( - TreeMap> prioritizedServers, DataSegment segment + Int2ObjectRBTreeMap> prioritizedServers, + DataSegment segment ) { - final Map.Entry> priorityServers = prioritizedServers.pollFirstEntry(); - - if (priorityServers == null) { - return null; - } + return Iterables.getOnlyElement(pick(prioritizedServers, segment, 1), null); + } - final Set servers = priorityServers.getValue(); - switch (servers.size()) { - case 0: - throw new ISE("[%s] Something hella weird going on here. We should not be here", segment.getIdentifier()); - case 1: - return priorityServers.getValue().iterator().next(); - default: - return serverSelectorStrategy.pick(servers, segment); + @Override + public List pick( + Int2ObjectRBTreeMap> prioritizedServers, + DataSegment segment, + int numServersToPick + ) + { + List result = new ArrayList<>(numServersToPick); + for (Set priorityServers : prioritizedServers.values()) { + result.addAll(serverSelectorStrategy.pick(priorityServers, segment, numServersToPick - result.size())); + if (result.size() == numServersToPick) { + break; + } } + return result; } } diff --git a/server/src/main/java/io/druid/client/selector/ConnectionCountServerSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/ConnectionCountServerSelectorStrategy.java index 8fdf68479388..1c525267f7f9 100644 --- a/server/src/main/java/io/druid/client/selector/ConnectionCountServerSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/ConnectionCountServerSelectorStrategy.java @@ -19,11 +19,14 @@ package io.druid.client.selector; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import io.druid.timeline.DataSegment; import java.util.Collections; import java.util.Comparator; +import java.util.List; import java.util.Set; public class ConnectionCountServerSelectorStrategy implements ServerSelectorStrategy @@ -42,4 +45,15 @@ public QueryableDruidServer pick(Set servers, DataSegment { return Collections.min(servers, comparator); } + + @Override + public List pick( + Set servers, DataSegment segment, int numServersToPick + ) + { + if (servers.size() <= numServersToPick) { + return ImmutableList.copyOf(servers); + } + return Ordering.from(comparator).leastOf(servers, numServersToPick); + } } diff --git a/server/src/main/java/io/druid/client/selector/RandomServerSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/RandomServerSelectorStrategy.java index 91360e93b0d5..439c68945942 100644 --- a/server/src/main/java/io/druid/client/selector/RandomServerSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/RandomServerSelectorStrategy.java @@ -19,19 +19,32 @@ package io.druid.client.selector; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; import io.druid.timeline.DataSegment; -import java.util.Random; +import java.util.Collections; +import java.util.List; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; public class RandomServerSelectorStrategy implements ServerSelectorStrategy { - private static final Random random = new Random(); - @Override public QueryableDruidServer pick(Set servers, DataSegment segment) { - return Iterators.get(servers.iterator(), random.nextInt(servers.size())); + return Iterators.get(servers.iterator(), ThreadLocalRandom.current().nextInt(servers.size())); + } + + @Override + public List pick(Set servers, DataSegment segment, int numServersToPick) + { + if (servers.size() <= numServersToPick) { + return ImmutableList.copyOf(servers); + } + List list = Lists.newArrayList(servers); + Collections.shuffle(list, ThreadLocalRandom.current()); + return ImmutableList.copyOf(list.subList(0, numServersToPick)); } } diff --git a/server/src/main/java/io/druid/client/selector/ServerSelector.java b/server/src/main/java/io/druid/client/selector/ServerSelector.java index 3236c4f88292..d04b66bf1b26 100644 --- a/server/src/main/java/io/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/io/druid/client/selector/ServerSelector.java @@ -19,27 +19,23 @@ package io.druid.client.selector; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.metamx.emitter.EmittingLogger; import io.druid.server.coordination.DruidServerMetadata; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; +import java.util.Collection; +import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.TreeMap; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; /** */ public class ServerSelector implements DiscoverySelector { - private static final EmittingLogger log = new EmittingLogger(ServerSelector.class); - - private final Set servers = Sets.newHashSet(); + private final Int2ObjectRBTreeMap> servers; private final TierSelectorStrategy strategy; @@ -50,8 +46,9 @@ public ServerSelector( TierSelectorStrategy strategy ) { - this.segment = new AtomicReference(segment); + this.segment = new AtomicReference<>(segment); this.strategy = strategy; + this.servers = new Int2ObjectRBTreeMap<>(strategy.getComparator()); } public DataSegment getSegment() @@ -65,14 +62,25 @@ public void addServerAndUpdateSegment( { synchronized (this) { this.segment.set(segment); - servers.add(server); + int priority = server.getServer().getPriority(); + Set priorityServers = servers.computeIfAbsent(priority, p -> new HashSet<>()); + priorityServers.add(server); } } public boolean removeServer(QueryableDruidServer server) { synchronized (this) { - return servers.remove(server); + int priority = server.getServer().getPriority(); + Set priorityServers = servers.get(priority); + if (priorityServers == null) { + return false; + } + boolean result = priorityServers.remove(server); + if (priorityServers.isEmpty()) { + servers.remove(priority); + } + return result; } } @@ -84,48 +92,28 @@ public boolean isEmpty() } public List getCandidates(final int numCandidates) { - List result = Lists.newArrayList(); synchronized (this) { - final DataSegment target = segment.get(); - for (Map.Entry> entry : toPrioritizedServers().entrySet()) { - Set servers = entry.getValue(); - TreeMap> tieredMap = Maps.newTreeMap(); - while (!servers.isEmpty()) { - tieredMap.put(entry.getKey(), servers); // strategy.pick() removes entry - QueryableDruidServer server = strategy.pick(tieredMap, target); - if (server == null) { - // regard this as any server in tieredMap is not appropriate - break; - } - result.add(server.getServer().getMetadata()); - if (numCandidates > 0 && result.size() >= numCandidates) { - return result; - } - servers.remove(server); - } + if (numCandidates > 0) { + return strategy.pick(servers, segment.get(), numCandidates) + .stream() + .map(server -> server.getServer().getMetadata()) + .collect(Collectors.toList()); + } else { + // return all servers as candidates + return servers.values() + .stream() + .flatMap(Collection::stream) + .map(server -> server.getServer().getMetadata()) + .collect(Collectors.toList()); } } - return result; } + @Override public QueryableDruidServer pick() { synchronized (this) { - return strategy.pick(toPrioritizedServers(), segment.get()); - } - } - - private TreeMap> toPrioritizedServers() - { - final TreeMap> prioritizedServers = new TreeMap<>(strategy.getComparator()); - for (QueryableDruidServer server : servers) { - Set theServers = prioritizedServers.get(server.getServer().getPriority()); - if (theServers == null) { - theServers = Sets.newHashSet(); - prioritizedServers.put(server.getServer().getPriority(), theServers); - } - theServers.add(server); + return strategy.pick(servers, segment.get()); } - return prioritizedServers; } } diff --git a/server/src/main/java/io/druid/client/selector/ServerSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/ServerSelectorStrategy.java index 9caf7179f3db..506ead7787a3 100644 --- a/server/src/main/java/io/druid/client/selector/ServerSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/ServerSelectorStrategy.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.timeline.DataSegment; +import java.util.List; import java.util.Set; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RandomServerSelectorStrategy.class) @@ -32,5 +33,7 @@ }) public interface ServerSelectorStrategy { - public QueryableDruidServer pick(Set servers, DataSegment segment); + QueryableDruidServer pick(Set servers, DataSegment segment); + + List pick(Set servers, DataSegment segment, int numServersToPick); } diff --git a/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java index 47a9910b8a83..f433fc266b7c 100644 --- a/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java @@ -22,10 +22,11 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; import java.util.Comparator; +import java.util.List; import java.util.Set; -import java.util.TreeMap; /** */ @@ -37,7 +38,13 @@ }) public interface TierSelectorStrategy { - public Comparator getComparator(); + Comparator getComparator(); - public QueryableDruidServer pick(TreeMap> prioritizedServers, DataSegment segment); + QueryableDruidServer pick(Int2ObjectRBTreeMap> prioritizedServers, DataSegment segment); + + List pick( + Int2ObjectRBTreeMap> prioritizedServers, + DataSegment segment, + int numServersToPick + ); } diff --git a/server/src/main/java/io/druid/curator/CuratorModule.java b/server/src/main/java/io/druid/curator/CuratorModule.java index 19db9df6dfd9..ead3ab641029 100644 --- a/server/src/main/java/io/druid/curator/CuratorModule.java +++ b/server/src/main/java/io/druid/curator/CuratorModule.java @@ -22,7 +22,10 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; - +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.java.util.common.lifecycle.Lifecycle; +import io.druid.java.util.common.logger.Logger; import org.apache.curator.ensemble.EnsembleProvider; import org.apache.curator.ensemble.exhibitor.DefaultExhibitorRestClient; import org.apache.curator.ensemble.exhibitor.ExhibitorEnsembleProvider; @@ -39,11 +42,6 @@ import java.io.IOException; import java.util.List; -import io.druid.guice.JsonConfigProvider; -import io.druid.guice.LazySingleton; -import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.java.util.common.logger.Logger; - /** */ public class CuratorModule implements Module @@ -147,7 +145,7 @@ public String getBackupConnectionString() throws Exception }; } - class SecuredACLProvider implements ACLProvider + static class SecuredACLProvider implements ACLProvider { @Override public List getDefaultAcl() diff --git a/server/src/main/java/io/druid/curator/announcement/Announcer.java b/server/src/main/java/io/druid/curator/announcement/Announcer.java index 2e5182f5922e..e042f6d77806 100644 --- a/server/src/main/java/io/druid/curator/announcement/Announcer.java +++ b/server/src/main/java/io/druid/curator/announcement/Announcer.java @@ -259,6 +259,11 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th } } break; + case INITIALIZED: + case CHILD_ADDED: + case CHILD_UPDATED: + case CONNECTION_SUSPENDED: + // do nothing } } } diff --git a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index 5260b0c919ec..78efbece1497 100644 --- a/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -256,9 +256,8 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th inventoryCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); strategy.newContainer(container); } - - break; } + break; case CHILD_REMOVED: synchronized (lock) { final ChildData child = event.getData(); @@ -281,9 +280,8 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th synchronized (removed) { markInventoryInitialized(removed); } - - break; } + break; case CHILD_UPDATED: synchronized (lock) { final ChildData child = event.getData(); @@ -310,9 +308,8 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th holder.setContainer(strategy.updateContainer(holder.getContainer(), container)); } } - - break; } + break; case INITIALIZED: synchronized (lock) { // must await initialized of all containerholders @@ -325,8 +322,12 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th } containersInitialized = true; maybeDoneInitializing(); - break; } + break; + case CONNECTION_SUSPENDED: + case CONNECTION_RECONNECTED: + case CONNECTION_LOST: + // do nothing } } @@ -431,7 +432,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th break; } - case INITIALIZED: + case INITIALIZED: { // make sure to acquire locks in (lock -> holder) order synchronized (lock) { synchronized (holder) { @@ -440,6 +441,11 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th } break; + } + case CONNECTION_SUSPENDED: + case CONNECTION_RECONNECTED: + case CONNECTION_LOST: + // do nothing } } } diff --git a/server/src/main/java/io/druid/guice/AnnouncerModule.java b/server/src/main/java/io/druid/guice/AnnouncerModule.java index 8d1152ea7cad..838780503935 100644 --- a/server/src/main/java/io/druid/guice/AnnouncerModule.java +++ b/server/src/main/java/io/druid/guice/AnnouncerModule.java @@ -25,8 +25,10 @@ import io.druid.concurrent.Execs; import io.druid.curator.announcement.Announcer; import io.druid.server.coordination.BatchDataSegmentAnnouncer; +import io.druid.server.coordination.CuratorDataSegmentServerAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncerProvider; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.curator.framework.CuratorFramework; @@ -40,7 +42,8 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); - binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + binder.bind(BatchDataSegmentAnnouncer.class).in(LazySingleton.class); + binder.bind(DataSegmentServerAnnouncer.class).to(CuratorDataSegmentServerAnnouncer.class).in(LazySingleton.class); } @Provides diff --git a/server/src/main/java/io/druid/guice/FirehoseModule.java b/server/src/main/java/io/druid/guice/FirehoseModule.java index 2c9fb462d1fa..4678edbfe4d1 100644 --- a/server/src/main/java/io/druid/guice/FirehoseModule.java +++ b/server/src/main/java/io/druid/guice/FirehoseModule.java @@ -28,9 +28,9 @@ import io.druid.segment.realtime.firehose.CombiningFirehoseFactory; import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import io.druid.segment.realtime.firehose.FixedCountFirehoseFactory; +import io.druid.segment.realtime.firehose.HttpFirehoseFactory; import io.druid.segment.realtime.firehose.IrcFirehoseFactory; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; -import io.druid.segment.realtime.firehose.ReplayableFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import java.util.Arrays; @@ -55,9 +55,9 @@ public List getJacksonModules() new NamedType(TimedShutoffFirehoseFactory.class, "timed"), new NamedType(IrcFirehoseFactory.class, "irc"), new NamedType(LocalFirehoseFactory.class, "local"), + new NamedType(HttpFirehoseFactory.class, "http"), new NamedType(EventReceiverFirehoseFactory.class, "receiver"), new NamedType(CombiningFirehoseFactory.class, "combining"), - new NamedType(ReplayableFirehoseFactory.class, "replayable"), new NamedType(FixedCountFirehoseFactory.class, "fixedCount") ) ); diff --git a/server/src/main/java/io/druid/guice/NodeTypeConfig.java b/server/src/main/java/io/druid/guice/NodeTypeConfig.java index 1ade9c07ab61..a01b47d53b2c 100644 --- a/server/src/main/java/io/druid/guice/NodeTypeConfig.java +++ b/server/src/main/java/io/druid/guice/NodeTypeConfig.java @@ -19,20 +19,22 @@ package io.druid.guice; +import io.druid.server.coordination.ServerType; + /** */ public class NodeTypeConfig { - private final String nodeType; + private final ServerType nodeType; public NodeTypeConfig( - String nodeType + ServerType nodeType ) { this.nodeType = nodeType; } - public String getNodeType() + public ServerType getNodeType() { return nodeType; } diff --git a/server/src/main/java/io/druid/guice/ParsersModule.java b/server/src/main/java/io/druid/guice/ParsersModule.java index 934083bcefbd..54c59ed0c351 100644 --- a/server/src/main/java/io/druid/guice/ParsersModule.java +++ b/server/src/main/java/io/druid/guice/ParsersModule.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; -import io.druid.data.input.ProtoBufInputRowParser; import io.druid.initialization.DruidModule; import io.druid.segment.realtime.firehose.IrcInputRowParser; @@ -45,7 +44,6 @@ public List getJacksonModules() return Arrays.asList( new SimpleModule("ParsersModule") .registerSubtypes( - new NamedType(ProtoBufInputRowParser.class, "protobuf"), new NamedType(IrcInputRowParser.class, "irc") ) ); diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index 0b7d84fe0035..3e4da18bb4d6 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -21,12 +21,13 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.multibindings.MapBinder; import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; -import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; @@ -63,6 +64,11 @@ */ public class QueryToolChestModule implements Module { + public static final String GENERIC_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.generic.queryMetricsFactory"; + public static final String GROUPBY_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.groupBy.queryMetricsFactory"; + public static final String TIMESERIES_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.timeseries.queryMetricsFactory"; + public static final String TOPN_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.topN.queryMetricsFactory"; + public final Map, Class> mappings = ImmutableMap., Class>builder() .put(TimeseriesQuery.class, TimeseriesQueryQueryToolChest.class) @@ -87,15 +93,54 @@ public void configure(Binder binder) binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); - binder.bind(GenericQueryMetricsFactory.class).to(DefaultGenericQueryMetricsFactory.class); - binder.bind(TopNQueryMetricsFactory.class).to(DefaultTopNQueryMetricsFactory.class); - binder.bind(GroupByQueryMetricsFactory.class).to(DefaultGroupByQueryMetricsFactory.class); - binder.bind(TimeseriesQueryMetricsFactory.class).to(DefaultTimeseriesQueryMetricsFactory.class); - JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.segmentMetadata", SegmentMetadataQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.select", SelectQueryConfig.class); + + PolyBind.createChoice( + binder, + GENERIC_QUERY_METRICS_FACTORY_PROPERTY, + Key.get(GenericQueryMetricsFactory.class), + Key.get(DefaultGenericQueryMetricsFactory.class) + ); + PolyBind + .optionBinder(binder, Key.get(GenericQueryMetricsFactory.class)) + .addBinding("default") + .to(DefaultGenericQueryMetricsFactory.class); + + PolyBind.createChoice( + binder, + GROUPBY_QUERY_METRICS_FACTORY_PROPERTY, + Key.get(GroupByQueryMetricsFactory.class), + Key.get(DefaultGroupByQueryMetricsFactory.class) + ); + PolyBind + .optionBinder(binder, Key.get(GroupByQueryMetricsFactory.class)) + .addBinding("default") + .to(DefaultGroupByQueryMetricsFactory.class); + + PolyBind.createChoice( + binder, + TIMESERIES_QUERY_METRICS_FACTORY_PROPERTY, + Key.get(TimeseriesQueryMetricsFactory.class), + Key.get(DefaultTimeseriesQueryMetricsFactory.class) + ); + PolyBind + .optionBinder(binder, Key.get(TimeseriesQueryMetricsFactory.class)) + .addBinding("default") + .to(DefaultTimeseriesQueryMetricsFactory.class); + + PolyBind.createChoice( + binder, + TOPN_QUERY_METRICS_FACTORY_PROPERTY, + Key.get(TopNQueryMetricsFactory.class), + Key.get(DefaultTopNQueryMetricsFactory.class) + ); + PolyBind + .optionBinder(binder, Key.get(TopNQueryMetricsFactory.class)) + .addBinding("default") + .to(DefaultTopNQueryMetricsFactory.class); } } diff --git a/server/src/main/java/io/druid/guice/ServerModule.java b/server/src/main/java/io/druid/guice/ServerModule.java index e960ee40ca2e..44c5f73b2892 100644 --- a/server/src/main/java/io/druid/guice/ServerModule.java +++ b/server/src/main/java/io/druid/guice/ServerModule.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import com.google.inject.Provides; - import io.druid.guice.annotations.Self; import io.druid.initialization.DruidModule; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; @@ -36,7 +35,7 @@ import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec; -import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -61,7 +60,7 @@ public ScheduledExecutorFactory getScheduledExecutorFactory(Lifecycle lifecycle) @Override public List getJacksonModules() { - return Arrays.asList( + return Collections.singletonList( new SimpleModule() .registerSubtypes( new NamedType(SingleDimensionShardSpec.class, "single"), diff --git a/server/src/main/java/io/druid/guice/ServerViewModule.java b/server/src/main/java/io/druid/guice/ServerViewModule.java index e88b1dbc55fd..d2860d107734 100644 --- a/server/src/main/java/io/druid/guice/ServerViewModule.java +++ b/server/src/main/java/io/druid/guice/ServerViewModule.java @@ -23,6 +23,7 @@ import com.google.inject.Module; import io.druid.client.FilteredServerInventoryView; import io.druid.client.FilteredServerInventoryViewProvider; +import io.druid.client.HttpServerInventoryViewConfig; import io.druid.client.InventoryView; import io.druid.client.ServerInventoryView; import io.druid.client.ServerInventoryViewProvider; @@ -37,6 +38,7 @@ public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.announcer", ServerInventoryViewProvider.class); JsonConfigProvider.bind(binder, "druid.announcer", FilteredServerInventoryViewProvider.class); + JsonConfigProvider.bind(binder, "druid.announcer.http", HttpServerInventoryViewConfig.class); binder.bind(InventoryView.class).to(ServerInventoryView.class); binder.bind(ServerView.class).to(ServerInventoryView.class); binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); diff --git a/server/src/main/java/io/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java b/server/src/main/java/io/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java index d3eed4134e65..962aa3ccb542 100644 --- a/server/src/main/java/io/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java +++ b/server/src/main/java/io/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java @@ -48,6 +48,7 @@ public SQLMetadataStorageUpdaterJobHandler(SQLMetadataConnector connector) this.dbi = connector.getDBI(); } + @Override public void publishSegments(final String tableName, final List segments, final ObjectMapper mapper) { dbi.withHandle( diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 08431383e599..9cc6fa41de03 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -275,6 +275,7 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi * * @return set of segments actually added */ + @Override public Set announceHistoricalSegments(final Set segments) throws IOException { final SegmentPublishResult result = announceHistoricalSegments(segments, null, null); @@ -315,7 +316,7 @@ public SegmentPublishResult announceHistoricalSegments( // Find which segments are used (i.e. not overshadowed). final Set usedSegments = Sets.newHashSet(); for (TimelineObjectHolder holder : VersionedIntervalTimeline.forSegments(segments) - .lookup(JodaUtils.ETERNITY)) { + .lookupWithIncompletePartitions(JodaUtils.ETERNITY)) { for (PartitionChunk chunk : holder.getObject()) { usedSegments.add(chunk.getObject()); } @@ -657,6 +658,7 @@ private boolean segmentExists(final Handle handle, final DataSegment segment) /** * Read dataSource metadata. Returns null if there is no metadata. */ + @Override public DataSourceMetadata getDataSourceMetadata(final String dataSource) { final byte[] bytes = connector.lookup( @@ -798,6 +800,7 @@ protected DataSourceMetadataUpdateResult updateDataSourceMetadataWithHandle( return retVal; } + @Override public boolean deleteDataSourceMetadata(final String dataSource) { return connector.retryWithHandle( @@ -853,6 +856,7 @@ public Boolean withHandle(Handle handle) throws Exception ); } + @Override public void updateSegmentMetadata(final Set segments) throws IOException { connector.getDBI().inTransaction( @@ -871,6 +875,7 @@ public Void inTransaction(Handle handle, TransactionStatus transactionStatus) th ); } + @Override public void deleteSegments(final Set segments) throws IOException { connector.getDBI().inTransaction( diff --git a/server/src/main/java/io/druid/metadata/MetadataRuleManagerProvider.java b/server/src/main/java/io/druid/metadata/MetadataRuleManagerProvider.java index 0ab82537ba95..c6b7922816a8 100644 --- a/server/src/main/java/io/druid/metadata/MetadataRuleManagerProvider.java +++ b/server/src/main/java/io/druid/metadata/MetadataRuleManagerProvider.java @@ -25,5 +25,6 @@ */ public interface MetadataRuleManagerProvider extends Provider { + @Override public MetadataRuleManager get(); } diff --git a/server/src/main/java/io/druid/metadata/MetadataSegmentManagerProvider.java b/server/src/main/java/io/druid/metadata/MetadataSegmentManagerProvider.java index 4fabab54fe09..aef060a2cba0 100644 --- a/server/src/main/java/io/druid/metadata/MetadataSegmentManagerProvider.java +++ b/server/src/main/java/io/druid/metadata/MetadataSegmentManagerProvider.java @@ -25,5 +25,6 @@ */ public interface MetadataSegmentManagerProvider extends Provider { + @Override public MetadataSegmentManager get(); } diff --git a/server/src/main/java/io/druid/metadata/MetadataSegmentPublisherProvider.java b/server/src/main/java/io/druid/metadata/MetadataSegmentPublisherProvider.java index 307dc4ba9aff..fdf3a688e8a2 100644 --- a/server/src/main/java/io/druid/metadata/MetadataSegmentPublisherProvider.java +++ b/server/src/main/java/io/druid/metadata/MetadataSegmentPublisherProvider.java @@ -21,5 +21,6 @@ public interface MetadataSegmentPublisherProvider extends SegmentPublisherProvider { + @Override public MetadataSegmentPublisher get(); } diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java index a9beb514950f..b3c1433679a2 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java @@ -444,6 +444,7 @@ public Void inTransaction(Handle handle, TransactionStatus transactionStatus) th public abstract DBI getDBI(); + @Override public void createDataSourceTable() { if (config.get().isCreateTables()) { @@ -633,6 +634,7 @@ public void createAuditTable() } } + @Override public void deleteAllRecords(final String tableName) { try { diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataRuleManager.java b/server/src/main/java/io/druid/metadata/SQLMetadataRuleManager.java index a9605e7fec14..a6e5d01a1331 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataRuleManager.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataRuleManager.java @@ -172,6 +172,7 @@ public SQLMetadataRuleManager( ); } + @Override @LifecycleStart public void start() { @@ -206,6 +207,7 @@ public void run() } } + @Override @LifecycleStop public void stop() { @@ -224,6 +226,7 @@ public void stop() } } + @Override public void poll() { try { @@ -314,17 +317,20 @@ public Map> fold( } } + @Override public Map> getAllRules() { return rules.get(); } + @Override public List getRules(final String dataSource) { List retVal = rules.get().get(dataSource); return retVal == null ? Lists.newArrayList() : retVal; } + @Override public List getRulesWithDefault(final String dataSource) { List retVal = Lists.newArrayList(); @@ -338,6 +344,7 @@ public List getRulesWithDefault(final String dataSource) return retVal; } + @Override public boolean overrideRule(final String dataSource, final List newRules, final AuditInfo auditInfo) { final String ruleString; diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java index 85a81479b10d..9388c78a7ab3 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java @@ -111,6 +111,7 @@ public SQLMetadataSegmentManager( this.connector = connector; } + @Override @LifecycleStart public void start() { @@ -145,6 +146,7 @@ public void run() } } + @Override @LifecycleStop public void stop() { diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java index 186ffc531b76..ac2c3df8ed02 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java @@ -87,6 +87,7 @@ public SQLMetadataStorageActionHandler( this.lockTable = lockTable; } + @Override public void insert( final String id, final DateTime timestamp, @@ -144,6 +145,7 @@ public boolean apply(Throwable e) } } + @Override public boolean setStatus(final String entryId, final boolean active, final StatusType status) { return connector.retryWithHandle( @@ -167,6 +169,7 @@ public Boolean withHandle(Handle handle) throws Exception ); } + @Override public Optional getEntry(final String entryId) { return connector.retryWithHandle( @@ -191,6 +194,7 @@ public Optional withHandle(Handle handle) throws Exception } + @Override public Optional getStatus(final String entryId) { return connector.retryWithHandle( @@ -214,6 +218,7 @@ public Optional withHandle(Handle handle) throws Exception ); } + @Override public List> getActiveEntriesWithStatus() { return connector.retryWithHandle( @@ -261,6 +266,7 @@ public Pair map(int index, ResultSet r, StatementContext } + @Override public List getInactiveStatusesSince(final DateTime timestamp) { return connector.retryWithHandle( @@ -302,6 +308,7 @@ public StatusType map(int index, ResultSet r, StatementContext ctx) throws SQLEx ); } + @Override public boolean addLock(final String entryId, final LockType lock) { return connector.retryWithHandle( @@ -324,6 +331,7 @@ public Boolean withHandle(Handle handle) throws Exception ); } + @Override public void removeLock(final long lockId) { connector.retryWithHandle( @@ -342,6 +350,7 @@ public Void withHandle(Handle handle) throws Exception ); } + @Override public boolean addLog(final String entryId, final LogType log) { return connector.retryWithHandle( @@ -364,6 +373,7 @@ public Boolean withHandle(Handle handle) throws Exception ); } + @Override public List getLogs(final String entryId) { return connector.retryWithHandle( @@ -413,6 +423,7 @@ public List fold( ); } + @Override public Map getLocks(final String entryId) { return connector.retryWithHandle( diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java index 920a099e0683..50037a5f7ef1 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandlerFactory.java @@ -40,6 +40,7 @@ public SQLMetadataStorageActionHandlerFactory( this.jsonMapper = jsonMapper; } + @Override public MetadataStorageActionHandler create( final String entryType, MetadataStorageActionHandlerTypes payloadTypes diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java b/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java index 1e3d8ddb760b..bf2bc580a7d1 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataSupervisorManager.java @@ -71,6 +71,7 @@ public SQLMetadataSupervisorManager( this.dbi = connector.getDBI(); } + @Override @LifecycleStart public void start() { diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index a071d0345714..99b639da3685 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -133,7 +133,6 @@ public RealtimeTuningConfig( this.reportParseExceptions = reportParseExceptions == null ? defaultReportParseExceptions : reportParseExceptions; - this.handoffConditionTimeout = handoffConditionTimeout == null ? defaultHandoffConditionTimeout : handoffConditionTimeout; @@ -143,6 +142,7 @@ public RealtimeTuningConfig( Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); } + @Override @JsonProperty public int getMaxRowsInMemory() { @@ -150,12 +150,6 @@ public int getMaxRowsInMemory() } @Override - public long getMaxPersistedSegmentsBytes() - { - // This option is not supported yet - throw new UnsupportedOperationException(); - } - @JsonProperty public Period getIntermediatePersistPeriod() { @@ -168,6 +162,7 @@ public Period getWindowPeriod() return windowPeriod; } + @Override @JsonProperty public File getBasePersistDirectory() { @@ -186,6 +181,7 @@ public RejectionPolicyFactory getRejectionPolicyFactory() return rejectionPolicyFactory; } + @Override @JsonProperty public int getMaxPendingPersists() { @@ -198,6 +194,7 @@ public ShardSpec getShardSpec() return shardSpec; } + @Override @JsonProperty public IndexSpec getIndexSpec() { @@ -222,6 +219,7 @@ public int getMergeThreadPriority() return this.mergeThreadPriority; } + @Override @JsonProperty public boolean isReportParseExceptions() { diff --git a/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java b/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java index 610329eaa7d4..bc3aee77958e 100644 --- a/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java +++ b/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java @@ -179,6 +179,7 @@ public String toString() '}'; } + @Override public GranularitySpec withIntervals(List inputIntervals) { return new UniformGranularitySpec(segmentGranularity, queryGranularity, rollup, inputIntervals); } diff --git a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java index 096e7dd07987..91bd6bf84570 100644 --- a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java +++ b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java @@ -33,7 +33,9 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.nio.file.FileAlreadyExistsException; +import java.util.Map; import java.util.UUID; /** @@ -73,7 +75,7 @@ public String getPathForHadoop(String dataSource) @Override public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException { - final String storageDir = DataSegmentPusherUtil.getStorageDir(segment); + final String storageDir = this.getStorageDir(segment); final File baseStorageDir = config.getStorageDirectory(); final File outDir = new File(baseStorageDir, storageDir); @@ -86,7 +88,7 @@ public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOExce } return createDescriptorFile( - segment.withLoadSpec(makeLoadSpec(outDir)) + segment.withLoadSpec(makeLoadSpec(outDir.toURI())) .withSize(size) .withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)), outDir @@ -98,7 +100,7 @@ public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOExce final long size = compressSegment(dataSegmentFile, tmpOutDir); final DataSegment dataSegment = createDescriptorFile( - segment.withLoadSpec(makeLoadSpec(new File(outDir, "index.zip"))) + segment.withLoadSpec(makeLoadSpec(new File(outDir, "index.zip").toURI())) .withSize(size) .withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)), tmpOutDir @@ -118,6 +120,12 @@ public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOExce return dataSegment; } + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + return ImmutableMap.of("type", "local", "path", finalIndexZipFilePath.getPath()); + } + private String intermediateDirFor(String storageDir) { return "intermediate_pushes/" + storageDir + "." + UUID.randomUUID().toString(); @@ -138,9 +146,4 @@ private DataSegment createDescriptorFile(DataSegment segment, File outDir) throw Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); return segment; } - - private ImmutableMap makeLoadSpec(File outFile) - { - return ImmutableMap.of("type", "local", "path", outFile.toString()); - } } diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java index 4995c3230274..19a97f7816a2 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java @@ -91,7 +91,7 @@ public boolean isSegmentLoaded(final DataSegment segment) public StorageLocation findStorageLocationIfLoaded(final DataSegment segment) { for (StorageLocation location : getSortedList(locations)) { - File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + File localStorageDir = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment)); if (localStorageDir.exists()) { return location; } @@ -124,7 +124,7 @@ public Segment getSegment(DataSegment segment) throws SegmentLoadingException public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { StorageLocation loc = findStorageLocationIfLoaded(segment); - String storageDir = DataSegmentPusherUtil.getStorageDir(segment); + String storageDir = DataSegmentPusher.getDefaultStorageDir(segment); if (loc == null) { loc = loadSegmentWithRetry(segment, storageDir); @@ -233,11 +233,11 @@ public void cleanup(DataSegment segment) throws SegmentLoadingException // in this case, findStorageLocationIfLoaded() will think segment is located in the failed storageDir which is actually not. // So we should always clean all possible locations here for (StorageLocation location : getSortedList(locations)) { - File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + File localStorageDir = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment)); if (localStorageDir.exists()) { // Druid creates folders of the form dataSource/interval/version/partitionNum. // We need to clean up all these directories if they are all empty. - File cacheFile = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + File cacheFile = new File(location.getPath(), DataSegmentPusher.getDefaultStorageDir(segment)); cleanupCacheFiles(location.getPath(), cacheFile); location.removeSegment(segment); } diff --git a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java index 8114d903256b..eca78928eccf 100644 --- a/server/src/main/java/io/druid/segment/realtime/FireDepartment.java +++ b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java @@ -101,7 +101,7 @@ public boolean checkFirehoseV2() public Firehose connect() throws IOException { - return ioConfig.getFirehoseFactory().connect(dataSchema.getParser()); + return ioConfig.getFirehoseFactory().connect(dataSchema.getParser(), null); } public FirehoseV2 connect(Object metaData) throws IOException diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index f13752b6006c..15ac09712690 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -20,6 +20,7 @@ package io.druid.segment.realtime; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -50,6 +51,7 @@ import io.druid.segment.realtime.plumber.Committers; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.Plumbers; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import org.joda.time.Interval; import java.io.Closeable; @@ -66,6 +68,7 @@ public class RealtimeManager implements QuerySegmentWalker private final List fireDepartments; private final QueryRunnerFactoryConglomerate conglomerate; + private final DataSegmentServerAnnouncer serverAnnouncer; /** * key=data source name,value=mappings of partition number to FireChief @@ -75,29 +78,32 @@ public class RealtimeManager implements QuerySegmentWalker @Inject public RealtimeManager( List fireDepartments, - QueryRunnerFactoryConglomerate conglomerate + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentServerAnnouncer serverAnnouncer ) { - this.fireDepartments = fireDepartments; - this.conglomerate = conglomerate; - - this.chiefs = Maps.newHashMap(); + this(fireDepartments, conglomerate, serverAnnouncer, Maps.newHashMap()); } + @VisibleForTesting RealtimeManager( List fireDepartments, QueryRunnerFactoryConglomerate conglomerate, + DataSegmentServerAnnouncer serverAnnouncer, Map> chiefs ) { this.fireDepartments = fireDepartments; this.conglomerate = conglomerate; - this.chiefs = chiefs; + this.serverAnnouncer = serverAnnouncer; + this.chiefs = chiefs == null ? Maps.newHashMap() : Maps.newHashMap(chiefs); } @LifecycleStart public void start() throws IOException { + serverAnnouncer.announce(); + for (final FireDepartment fireDepartment : fireDepartments) { final DataSchema schema = fireDepartment.getDataSchema(); @@ -129,6 +135,8 @@ public void stop() CloseQuietly.close(chief); } } + + serverAnnouncer.unannounce(); } public FireDepartmentMetrics getMetrics(String datasource) @@ -303,14 +311,15 @@ public void run() catch (RuntimeException e) { log.makeAlert( e, - "RuntimeException aborted realtime processing[%s]", + "[%s] aborted realtime processing[%s]", + e.getClass().getSimpleName(), fireDepartment.getDataSchema().getDataSource() ).emit(); normalExit = false; - throw e; + throw Throwables.propagate(e); } catch (Error e) { - log.makeAlert(e, "Exception aborted realtime processing[%s]", fireDepartment.getDataSchema().getDataSource()) + log.makeAlert(e, "Error aborted realtime processing[%s]", fireDepartment.getDataSchema().getDataSource()) .emit(); normalExit = false; throw e; @@ -387,6 +396,7 @@ public QueryRunner getQueryRunner(Query query) return new FinalizeResultsQueryRunner(plumber.getQueryRunner(query), toolChest); } + @Override public void close() throws IOException { synchronized (this) { diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index 579f4f139040..45caa34d9af4 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -27,6 +27,7 @@ import io.druid.segment.incremental.IndexSizeExceededException; import java.io.Closeable; +import java.util.Collection; import java.util.List; /** @@ -52,14 +53,6 @@ public interface Appenderator extends QuerySegmentWalker, Closeable */ Object startJob(); - /** - * Returns the size of data currently stored in local storage. The persistedBytes is changed when - * {@link #persistAll(Committer)} and {@link #drop(SegmentIdentifier)} is called. - * - * @return the size of persisted data in bytes - */ - long getPersistedBytes(); - /** * Add a row. Must not be called concurrently from multiple threads. *

@@ -69,7 +62,7 @@ public interface Appenderator extends QuerySegmentWalker, Closeable * Committer is guaranteed to be *created* synchronously with the call to add, but will actually be used * asynchronously. *

- * The add, clear, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifier the segment into which this row should be added * @param row the row to add @@ -99,12 +92,19 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe */ int getRowCount(final SegmentIdentifier identifier); + /** + * Returns the number of total rows in this appenderator. + * + * @return total number of rows + */ + int getTotalRowCount(); + /** * Drop all in-memory and on-disk data, and forget any previously-remembered commit metadata. This could be useful if, * for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been * cleared. This may take some time, since all pending persists must finish first. * - * The add, clear, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread. */ void clear() throws InterruptedException; @@ -122,27 +122,47 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe */ ListenableFuture drop(SegmentIdentifier identifier); + /** + * Persist any in-memory indexed data for segments of the given identifiers to durable storage. This may be only + * somewhat durable, e.g. the machine's local disk. The Committer will be made synchronously with the call to + * persist, but will actually be used asynchronously. Any metadata returned by the committer will be associated with + * the data persisted to disk. + *

+ * The add, clear, persist, persistAll, and push methods should all be called from the same thread. + * + * @param identifiers segment identifiers to be persisted + * @param committer a committer associated with all data that has been added to segments of the given identifiers so + * far + * + * @return future that resolves when all pending data to segments of the identifiers has been persisted, contains + * commit metadata for this persist + */ + ListenableFuture persist(Collection identifiers, Committer committer); + /** * Persist any in-memory indexed data to durable storage. This may be only somewhat durable, e.g. the - * machine's local disk. The Committer will be made synchronously will the call to persistAll, but will actually + * machine's local disk. The Committer will be made synchronously with the call to persistAll, but will actually * be used asynchronously. Any metadata returned by the committer will be associated with the data persisted to * disk. *

- * The add, clear, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param committer a committer associated with all data that has been added so far * * @return future that resolves when all pending data has been persisted, contains commit metadata for this persist */ - ListenableFuture persistAll(Committer committer); + default ListenableFuture persistAll(Committer committer) + { + return persist(getSegments(), committer); + } /** - * Merge and push particular segments to deep storage. This will trigger an implicit {@link #persistAll(Committer)} - * using the provided Committer. + * Merge and push particular segments to deep storage. This will trigger an implicit + * {@link #persist(Collection, Committer)} using the provided Committer. *

* After this method is called, you cannot add new data to any segments that were previously under construction. *

- * The add, clear, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifiers list of segments to push * @param committer a committer associated with all data that has been added so far @@ -150,11 +170,12 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * @return future that resolves when all segments have been pushed. The segment list will be the list of segments * that have been pushed and the commit metadata from the Committer. */ - ListenableFuture push(List identifiers, Committer committer); + ListenableFuture push(Collection identifiers, Committer committer); /** * Stop any currently-running processing and clean up after ourselves. This will not remove any on-disk persisted * data, but it will drop any data that has not yet been persisted. */ + @Override void close(); } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java index dd6c6bd04b7b..4b2c99f644c0 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -30,8 +30,6 @@ public interface AppenderatorConfig int getMaxRowsInMemory(); - long getMaxPersistedSegmentsBytes(); - int getMaxPendingPersists(); Period getIntermediatePersistPeriod(); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java new file mode 100644 index 000000000000..ad0fa0a57c2b --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java @@ -0,0 +1,673 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; +import io.druid.concurrent.Execs; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.SegmentDescriptor; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +/** + * A AppenderatorDriver drives an Appenderator to index a finite stream of data. This class does not help you + * index unbounded streams. All handoff is done at the end of indexing. + * + * This class helps with doing things that Appenderators don't, including deciding which segments to use (with a + * SegmentAllocator), publishing segments to the metadata store (with a SegmentPublisher), and monitoring handoff (with + * a SegmentHandoffNotifier). + * + * Note that the commit metadata stored by this class via the underlying Appenderator is not the same metadata as + * you pass in. It's wrapped in some extra metadata needed by the driver. + */ +public class AppenderatorDriver implements Closeable +{ + private static final Logger log = new Logger(AppenderatorDriver.class); + + private final Appenderator appenderator; + private final SegmentAllocator segmentAllocator; + private final SegmentHandoffNotifier handoffNotifier; + private final UsedSegmentChecker usedSegmentChecker; + private final ObjectMapper objectMapper; + private final FireDepartmentMetrics metrics; + + // All access to "activeSegments", "publishPendingSegments", and "lastSegmentId" must be synchronized on + // "activeSegments". + + // sequenceName -> start of segment interval -> segment we're currently adding data to + private final Map> activeSegments = new TreeMap<>(); + + // sequenceName -> list of identifiers of segments waiting for being published + // publishPendingSegments is always a super set of activeSegments because there can be some segments to which data + // are not added anymore, but not published yet. + private final Map> publishPendingSegments = new HashMap<>(); + + // sequenceName -> most recently allocated segment + private final Map lastSegmentIds = Maps.newHashMap(); + + private final ListeningExecutorService publishExecutor; + + /** + * Create a driver. + * + * @param appenderator appenderator + * @param segmentAllocator segment allocator + * @param handoffNotifierFactory handoff notifier factory + * @param usedSegmentChecker used segment checker + * @param objectMapper object mapper, used for serde of commit metadata + * @param metrics Firedepartment metrics + */ + public AppenderatorDriver( + Appenderator appenderator, + SegmentAllocator segmentAllocator, + SegmentHandoffNotifierFactory handoffNotifierFactory, + UsedSegmentChecker usedSegmentChecker, + ObjectMapper objectMapper, + FireDepartmentMetrics metrics + ) + { + this.appenderator = Preconditions.checkNotNull(appenderator, "appenderator"); + this.segmentAllocator = Preconditions.checkNotNull(segmentAllocator, "segmentAllocator"); + this.handoffNotifier = Preconditions.checkNotNull(handoffNotifierFactory, "handoffNotifierFactory") + .createSegmentHandoffNotifier(appenderator.getDataSource()); + this.usedSegmentChecker = Preconditions.checkNotNull(usedSegmentChecker, "usedSegmentChecker"); + this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); + this.metrics = Preconditions.checkNotNull(metrics, "metrics"); + this.publishExecutor = MoreExecutors.listeningDecorator(Execs.singleThreaded("publish-%d")); + } + + /** + * Perform any initial setup and return currently persisted commit metadata. + * + * Note that this method returns the same metadata you've passed in with your Committers, even though this class + * stores extra metadata on disk. + * + * @return currently persisted commit metadata + */ + public Object startJob() + { + handoffNotifier.start(); + + final AppenderatorDriverMetadata metadata = objectMapper.convertValue( + appenderator.startJob(), + AppenderatorDriverMetadata.class + ); + + log.info("Restored metadata[%s].", metadata); + + if (metadata != null) { + synchronized (activeSegments) { + for (Map.Entry> entry : metadata.getActiveSegments().entrySet()) { + final String sequenceName = entry.getKey(); + final TreeMap segmentMap = Maps.newTreeMap(); + + activeSegments.put(sequenceName, segmentMap); + + for (SegmentIdentifier identifier : entry.getValue()) { + segmentMap.put(identifier.getInterval().getStartMillis(), identifier); + } + } + publishPendingSegments.putAll(metadata.getPublishPendingSegments()); + lastSegmentIds.putAll(metadata.getLastSegmentIds()); + } + + return metadata.getCallerMetadata(); + } else { + return null; + } + } + + private void addSegment(String sequenceName, SegmentIdentifier identifier) + { + synchronized (activeSegments) { + activeSegments.computeIfAbsent(sequenceName, k -> new TreeMap<>()) + .putIfAbsent(identifier.getInterval().getStartMillis(), identifier); + + publishPendingSegments.computeIfAbsent(sequenceName, k -> new ArrayList<>()) + .add(identifier); + lastSegmentIds.put(sequenceName, identifier.getIdentifierAsString()); + } + } + + /** + * Clears out all our state and also calls {@link Appenderator#clear()} on the underlying Appenderator. + */ + public void clear() throws InterruptedException + { + synchronized (activeSegments) { + activeSegments.clear(); + } + appenderator.clear(); + } + + /** + * Add a row. Must not be called concurrently from multiple threads. + * + * @param row the row to add + * @param sequenceName sequenceName for this row's segment + * @param committerSupplier supplier of a committer associated with all data that has been added, including this row + * + * @return segment to which this row was added, or null if segment allocator returned null for this row + * + * @throws IOException if there is an I/O error while allocating or writing to a segment + */ + public AppenderatorDriverAddResult add( + final InputRow row, + final String sequenceName, + final Supplier committerSupplier + ) throws IOException + { + Preconditions.checkNotNull(row, "row"); + Preconditions.checkNotNull(sequenceName, "sequenceName"); + Preconditions.checkNotNull(committerSupplier, "committerSupplier"); + + final SegmentIdentifier identifier = getSegment(row, sequenceName); + + if (identifier != null) { + try { + final int numRowsInMemory = appenderator.add(identifier, row, wrapCommitterSupplier(committerSupplier)); + return AppenderatorDriverAddResult.ok(identifier, numRowsInMemory, appenderator.getTotalRowCount()); + } + catch (SegmentNotWritableException e) { + throw new ISE(e, "WTF?! Segment[%s] not writable when it should have been.", identifier); + } + } else { + return AppenderatorDriverAddResult.fail(); + } + } + + /** + * Persist all data indexed through this driver so far. Blocks until complete. + * + * Should be called after all data has been added through {@link #add(InputRow, String, Supplier)}. + * + * @param committer committer representing all data that has been added so far + * + * @return commitMetadata persisted + */ + public Object persist(final Committer committer) throws InterruptedException + { + try { + log.info("Persisting data."); + final long start = System.currentTimeMillis(); + final Object commitMetadata = appenderator.persistAll(wrapCommitter(committer)).get(); + log.info("Persisted pending data in %,dms.", System.currentTimeMillis() - start); + return commitMetadata; + } + catch (InterruptedException e) { + throw e; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Register the segments in the given {@link SegmentsAndMetadata} to be handed off and execute a background task which + * waits until the hand off completes. + * + * @param segmentsAndMetadata the result segments and metadata of + * {@link #publish(TransactionalSegmentPublisher, Committer, Collection)} + * + * @return null if the input segmentsAndMetadata is null. Otherwise, a {@link ListenableFuture} for the submitted task + * which returns {@link SegmentsAndMetadata} containing the segments successfully handed off and the metadata + * of the caller of {@link AppenderatorDriverMetadata} + */ + public ListenableFuture registerHandoff(SegmentsAndMetadata segmentsAndMetadata) + { + if (segmentsAndMetadata == null) { + return Futures.immediateFuture(null); + + } else { + final List waitingSegmentIdList = segmentsAndMetadata.getSegments().stream() + .map(SegmentIdentifier::fromDataSegment) + .collect(Collectors.toList()); + + if (waitingSegmentIdList.isEmpty()) { + return Futures.immediateFuture( + new SegmentsAndMetadata( + segmentsAndMetadata.getSegments(), + ((AppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()) + .getCallerMetadata() + ) + ); + } + + log.info("Register handoff of segments: [%s]", waitingSegmentIdList); + + final SettableFuture resultFuture = SettableFuture.create(); + final AtomicInteger numRemainingHandoffSegments = new AtomicInteger(waitingSegmentIdList.size()); + + for (final SegmentIdentifier segmentIdentifier : waitingSegmentIdList) { + handoffNotifier.registerSegmentHandoffCallback( + new SegmentDescriptor( + segmentIdentifier.getInterval(), + segmentIdentifier.getVersion(), + segmentIdentifier.getShardSpec().getPartitionNum() + ), + MoreExecutors.sameThreadExecutor(), + () -> { + log.info("Segment[%s] successfully handed off, dropping.", segmentIdentifier); + metrics.incrementHandOffCount(); + + final ListenableFuture dropFuture = appenderator.drop(segmentIdentifier); + Futures.addCallback( + dropFuture, + new FutureCallback() + { + @Override + public void onSuccess(Object result) + { + if (numRemainingHandoffSegments.decrementAndGet() == 0) { + log.info("All segments handed off."); + resultFuture.set( + new SegmentsAndMetadata( + segmentsAndMetadata.getSegments(), + ((AppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()) + .getCallerMetadata() + ) + ); + } + } + + @Override + public void onFailure(Throwable e) + { + log.warn(e, "Failed to drop segment[%s]?!", segmentIdentifier); + numRemainingHandoffSegments.decrementAndGet(); + resultFuture.setException(e); + } + } + ); + } + ); + } + + return resultFuture; + } + } + + /** + * Closes this driver. Does not close the underlying Appenderator; you should do that yourself. + */ + @Override + public void close() + { + publishExecutor.shutdownNow(); + handoffNotifier.close(); + } + + private SegmentIdentifier getActiveSegment(final DateTime timestamp, final String sequenceName) + { + synchronized (activeSegments) { + final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); + + if (activeSegmentsForSequence == null) { + return null; + } + + final Map.Entry candidateEntry = activeSegmentsForSequence.floorEntry(timestamp.getMillis()); + if (candidateEntry != null && candidateEntry.getValue().getInterval().contains(timestamp)) { + return candidateEntry.getValue(); + } else { + return null; + } + } + } + + /** + * Return a segment usable for "timestamp". May return null if no segment can be allocated. + * + * @param timestamp data timestamp + * @param sequenceName sequenceName for potential segment allocation + * + * @return identifier, or null + * + * @throws IOException if an exception occurs while allocating a segment + */ + private SegmentIdentifier getSegment(final InputRow row, final String sequenceName) throws IOException + { + final SegmentIdentifier existing = getActiveSegment(row.getTimestamp(), sequenceName); + if (existing != null) { + return existing; + } else { + // Allocate new segment. + final String previousSegmentId; + synchronized (activeSegments) { + previousSegmentId = lastSegmentIds.get(sequenceName); + } + final SegmentIdentifier newSegment = segmentAllocator.allocate( + row, + sequenceName, + previousSegmentId + ); + + if (newSegment != null) { + for (SegmentIdentifier identifier : appenderator.getSegments()) { + if (identifier.equals(newSegment)) { + throw new ISE( + "WTF?! Allocated segment[%s] which conflicts with existing segment[%s].", + newSegment, + identifier + ); + } + } + + log.info("New segment[%s] for sequenceName[%s].", newSegment, sequenceName); + addSegment(sequenceName, newSegment); + } else { + // Well, we tried. + log.warn("Cannot allocate segment for timestamp[%s], sequenceName[%s]. ", row.getTimestamp(), sequenceName); + } + + return newSegment; + } + } + + /** + * Move a set of identifiers out from "active", making way for newer segments. + */ + public boolean moveSegmentOut( + final String sequenceName, + final List identifiers + ) + { + synchronized (activeSegments) { + final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); + if (activeSegmentsForSequence == null) { + throw new ISE("WTF?! Asked to remove segments for sequenceName[%s] which doesn't exist...", sequenceName); + } + + for (final SegmentIdentifier identifier : identifiers) { + log.info("Moving segment[%s] out of active list.", identifier); + final long key = identifier.getInterval().getStartMillis(); + if (!activeSegmentsForSequence.remove(key).equals(identifier)) { + throw new ISE("WTF?! Asked to remove segment[%s] that didn't exist...", identifier); + } + } + return true; + } + } + + public ListenableFuture publishAll( + final TransactionalSegmentPublisher publisher, + final Committer committer + ) + { + final List sequenceNames; + synchronized (activeSegments) { + sequenceNames = ImmutableList.copyOf(publishPendingSegments.keySet()); + } + return publish(publisher, committer, sequenceNames); + } + + /** + * Execute a task in background to publish all segments corresponding to the given sequence names. The task + * internally pushes the segments to the deep storage first, and then publishes the metadata to the metadata storage. + * + * @param publisher segment publisher + * @param committer committer + * @param sequenceNames a collection of sequence names to be published + * + * @return a {@link ListenableFuture} for the submitted task which removes published {@code sequenceNames} from + * {@code activeSegments} and {@code publishPendingSegments}. + */ + public ListenableFuture publish( + final TransactionalSegmentPublisher publisher, + final Committer committer, + final Collection sequenceNames + ) + { + final List theSegments; + synchronized (activeSegments) { + theSegments = sequenceNames.stream() + .map(publishPendingSegments::get) + .filter(Objects::nonNull) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + final ListenableFuture publishFuture = publish( + publisher, + wrapCommitter(committer), + theSegments + ); + + Futures.addCallback( + publishFuture, + new FutureCallback() + { + @Override + public void onSuccess(@Nullable SegmentsAndMetadata result) + { + if (result != null) { + synchronized (activeSegments) { + // Remove sequenceName from both publishPendingSemgments and activeSegments + sequenceNames.forEach( + sequenceName -> { + activeSegments.remove(sequenceName); + publishPendingSegments.remove(sequenceName); + } + ); + } + } + } + + @Override + public void onFailure(Throwable t) + { + // The throwable is propagated anyway when get() is called on the future. + // See FiniteAppenderatorFailTest.testInterruptDuringPush(). + log.error(t, "Failed to publish segments[%s]", theSegments); + } + } + ); + + return publishFuture; + } + + /** + * Execute a task in background to publish the given segments. The task blocks until complete. + * Retries forever on transient failures, but may exit early on permanent failures. + * + * Should be called after all data has been added through {@link #add(InputRow, String, Supplier)}. + * + * @param publisher publisher to use for this set of segments + * @param wrappedCommitter committer representing all data that has been added so far + * + * @return segments and metadata published if successful, or null if segments could not be handed off due to + * transaction failure with commit metadata. + */ + private ListenableFuture publish( + final TransactionalSegmentPublisher publisher, + final WrappedCommitter wrappedCommitter, + final List segmentIdentifiers + ) + { + return publishExecutor.submit( + () -> { + long nTry = 0; + while (true) { + try { + log.info("Pushing segments: [%s]", Joiner.on(", ").join(segmentIdentifiers)); + final SegmentsAndMetadata segmentsAndMetadata = appenderator.push(segmentIdentifiers, wrappedCommitter) + .get(); + + // Sanity check + final Set pushedSegments = segmentsAndMetadata.getSegments().stream() + .map(SegmentIdentifier::fromDataSegment) + .collect(Collectors.toSet()); + if (!pushedSegments.equals(Sets.newHashSet(segmentIdentifiers))) { + throw new ISE( + "WTF?! Pushed different segments than requested. Pushed[%s], requested[%s].", + pushedSegments, + segmentIdentifiers + ); + } + + log.info( + "Publishing segments with commitMetadata[%s]: [%s]", + segmentsAndMetadata.getCommitMetadata(), + Joiner.on(", ").join(segmentsAndMetadata.getSegments()) + ); + + if (segmentsAndMetadata.getSegments().isEmpty()) { + log.info("Nothing to publish, skipping publish step."); + } else { + final boolean published = publisher.publishSegments( + ImmutableSet.copyOf(segmentsAndMetadata.getSegments()), + ((AppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() + ); + + if (published) { + log.info("Published segments, awaiting handoff."); + } else { + log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); + if (usedSegmentChecker.findUsedSegments(pushedSegments) + .equals(Sets.newHashSet(segmentsAndMetadata.getSegments()))) { + log.info("Our segments really do exist, awaiting handoff."); + } else { + log.warn("Our segments don't exist, giving up."); + return null; + } + } + } + + return segmentsAndMetadata; + } + catch (InterruptedException e) { + throw e; + } + catch (Exception e) { + final long sleepMillis = computeNextRetrySleep(++nTry); + log.warn(e, "Failed publish (try %d), retrying in %,dms.", nTry, sleepMillis); + Thread.sleep(sleepMillis); + } + } + } + ); + } + + public ListenableFuture publishAndRegisterHandoff( + final TransactionalSegmentPublisher publisher, + final Committer committer, + final Collection sequenceNames + ) + { + return Futures.transform( + publish(publisher, committer, sequenceNames), + this::registerHandoff + ); + } + + private interface WrappedCommitter extends Committer + { + } + + private Supplier wrapCommitterSupplier(final Supplier committerSupplier) + { + return () -> wrapCommitter(committerSupplier.get()); + } + + private WrappedCommitter wrapCommitter(final Committer committer) + { + final AppenderatorDriverMetadata wrappedMetadata; + synchronized (activeSegments) { + wrappedMetadata = new AppenderatorDriverMetadata( + ImmutableMap.copyOf( + Maps.transformValues( + activeSegments, + new Function, List>() + { + @Override + public List apply(NavigableMap input) + { + return ImmutableList.copyOf(input.values()); + } + } + ) + ), + ImmutableMap.copyOf(publishPendingSegments), + ImmutableMap.copyOf(lastSegmentIds), + committer.getMetadata() + ); + } + + return new WrappedCommitter() + { + @Override + public Object getMetadata() + { + return wrappedMetadata; + } + + @Override + public void run() + { + committer.run(); + } + }; + } + + private static long computeNextRetrySleep(final long nTry) + { + final long baseSleepMillis = 1000; + final long maxSleepMillis = 60000; + final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * new Random().nextGaussian(), 0), 2); + return (long) (Math.min(maxSleepMillis, baseSleepMillis * Math.pow(2, nTry)) * fuzzyMultiplier); + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java new file mode 100644 index 000000000000..3e5d6f1fe6d3 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverAddResult.java @@ -0,0 +1,81 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.appenderator; + +import com.google.common.base.Supplier; +import io.druid.data.input.InputRow; + +import javax.annotation.Nullable; + +/** + * Result of {@link AppenderatorDriver#add(InputRow, String, Supplier)}. It contains the identifier of the + * segment which the InputRow is added to and the number of rows in that segment. + */ +public class AppenderatorDriverAddResult +{ + private final SegmentIdentifier segmentIdentifier; + private final int numRowsInSegment; + private final long totalNumRowsInAppenderator; + + public static AppenderatorDriverAddResult ok( + SegmentIdentifier segmentIdentifier, + int numRowsInSegment, + long persistedBytes + ) + { + return new AppenderatorDriverAddResult(segmentIdentifier, numRowsInSegment, persistedBytes); + } + + public static AppenderatorDriverAddResult fail() + { + return new AppenderatorDriverAddResult(null, 0, 0); + } + + private AppenderatorDriverAddResult( + @Nullable SegmentIdentifier segmentIdentifier, + int numRowsInSegment, + long totalNumRowsInAppenderator + ) + { + this.segmentIdentifier = segmentIdentifier; + this.numRowsInSegment = numRowsInSegment; + this.totalNumRowsInAppenderator = totalNumRowsInAppenderator; + } + + public boolean isOk() + { + return segmentIdentifier != null; + } + + public SegmentIdentifier getSegmentIdentifier() + { + return segmentIdentifier; + } + + public int getNumRowsInSegment() + { + return numRowsInSegment; + } + + public long getTotalNumRowsInAppenderator() + { + return totalNumRowsInAppenderator; + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java similarity index 78% rename from server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java rename to server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java index 65c7e70c7888..edff72572d9e 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriverMetadata.java @@ -25,20 +25,23 @@ import java.util.List; import java.util.Map; -public class FiniteAppenderatorDriverMetadata +public class AppenderatorDriverMetadata { private final Map> activeSegments; + private final Map> publishPendingSegments; private final Map lastSegmentIds; private final Object callerMetadata; @JsonCreator - public FiniteAppenderatorDriverMetadata( + public AppenderatorDriverMetadata( @JsonProperty("activeSegments") Map> activeSegments, + @JsonProperty("publishPendingSegments") Map> publishPendingSegments, @JsonProperty("lastSegmentIds") Map lastSegmentIds, @JsonProperty("callerMetadata") Object callerMetadata ) { this.activeSegments = activeSegments; + this.publishPendingSegments = publishPendingSegments; this.lastSegmentIds = lastSegmentIds; this.callerMetadata = callerMetadata; } @@ -49,6 +52,12 @@ public Map> getActiveSegments() return activeSegments; } + @JsonProperty + public Map> getPublishPendingSegments() + { + return publishPendingSegments; + } + @JsonProperty public Map getLastSegmentIds() { @@ -64,8 +73,9 @@ public Object getCallerMetadata() @Override public String toString() { - return "FiniteAppenderatorDriverMetadata{" + + return "AppenderatorDriverMetadata{" + "activeSegments=" + activeSegments + + ", publishPendingSegments=" + publishPendingSegments + ", lastSegmentIds=" + lastSegmentIds + ", callerMetadata=" + callerMetadata + '}'; diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index b1a189fc4a89..d780fa99bbbf 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -80,6 +80,7 @@ import java.nio.channels.FileLock; import java.nio.file.StandardOpenOption; import java.util.Arrays; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -88,7 +89,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; /** */ @@ -112,7 +112,7 @@ public class AppenderatorImpl implements Appenderator private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( String.CASE_INSENSITIVE_ORDER ); - private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); + private final QuerySegmentWalker texasRanger; private volatile ListeningExecutorService persistExecutor = null; @@ -121,7 +121,8 @@ public class AppenderatorImpl implements Appenderator private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; - private long persistedBytes; + private int rowsCurrentlyInMemory; + private int totalRows; public AppenderatorImpl( DataSchema schema, @@ -168,12 +169,6 @@ public String getDataSource() return schema.getDataSource(); } - @Override - public long getPersistedBytes() - { - return persistedBytes; - } - @Override public Object startJob() { @@ -220,11 +215,13 @@ public int add( throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier); } - rowsCurrentlyInMemory.addAndGet(sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd); + final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; + rowsCurrentlyInMemory += numAddedRows; + totalRows += numAddedRows; if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush - || rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { + || rowsCurrentlyInMemory >= tuningConfig.getMaxRowsInMemory()) { // persistAll clears rowsCurrentlyInMemory, no need to update it. persistAll(committerSupplier.get()); } @@ -250,10 +247,16 @@ public int getRowCount(final SegmentIdentifier identifier) } } + @Override + public int getTotalRowCount() + { + return totalRows; + } + @VisibleForTesting int getRowsInMemory() { - return rowsCurrentlyInMemory.get(); + return rowsCurrentlyInMemory; } private Sink getOrCreateSink(final SegmentIdentifier identifier) @@ -313,20 +316,22 @@ public void clear() throws InterruptedException // Drop commit metadata, then abandon all segments. try { - final ListenableFuture uncommitFuture = persistExecutor.submit( - new Callable() - { - @Override - public Object call() throws Exception + if (persistExecutor != null) { + final ListenableFuture uncommitFuture = persistExecutor.submit( + new Callable() { - objectMapper.writeValue(computeCommitFile(), Committed.nil()); - return null; + @Override + public Object call() throws Exception + { + objectMapper.writeValue(computeCommitFile(), Committed.nil()); + return null; + } } - } - ); + ); - // Await uncommit. - uncommitFuture.get(); + // Await uncommit. + uncommitFuture.get(); + } // Drop everything. final List> futures = Lists.newArrayList(); @@ -354,17 +359,16 @@ public ListenableFuture drop(final SegmentIdentifier identifier) } @Override - public ListenableFuture persistAll(final Committer committer) + public ListenableFuture persist(Collection identifiers, Committer committer) { - // Submit persistAll task to the persistExecutor - final Map commitHydrants = Maps.newHashMap(); final List> indexesToPersist = Lists.newArrayList(); - final Set identifiers = sinks.keySet(); + int numPersistedRows = 0; for (SegmentIdentifier identifier : identifiers) { final Sink sink = sinks.get(identifier); final List hydrants = Lists.newArrayList(sink); commitHydrants.put(identifier, hydrants.size()); + numPersistedRows += sink.getNumRowsInMemory(); final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); @@ -440,14 +444,21 @@ public String apply(Map.Entry entry) resetNextFlush(); // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. - rowsCurrentlyInMemory.set(0); + rowsCurrentlyInMemory -= numPersistedRows; return future; } + @Override + public ListenableFuture persistAll(final Committer committer) + { + // Submit persistAll task to the persistExecutor + return persist(sinks.keySet(), committer); + } + @Override public ListenableFuture push( - final List identifiers, + final Collection identifiers, final Committer committer ) { @@ -462,7 +473,7 @@ public ListenableFuture push( } return Futures.transform( - persistAll(committer), + persist(identifiers, committer), new Function() { @Override @@ -578,11 +589,9 @@ private DataSegment mergeAndPush(final SegmentIdentifier identifier, final Sink tuningConfig.getIndexSpec() ); - QueryableIndex index = indexIO.loadIndex(mergedFile); - DataSegment segment = dataSegmentPusher.push( mergedFile, - sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) + sink.getSegment().withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes)) ); objectMapper.writeValue(descriptorFile, segment); @@ -870,7 +879,8 @@ private ListenableFuture abandonSegment( droppingSinks.add(identifier); // Decrement this sink's rows from rowsCurrentlyInMemory (we only count active sinks). - rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); + rowsCurrentlyInMemory -= sink.getNumRowsInMemory(); + totalRows -= sink.getNumRows(); // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( @@ -928,19 +938,18 @@ public Object apply(@Nullable Object input) if (cache != null) { cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); } + try { + hydrant.getSegment().close(); + } + catch (IOException e) { + log.makeAlert(e, "Failed to explicitly close segment[%s]", schema.getDataSource()) + .addData("identifier", hydrant.getSegment().getIdentifier()) + .emit(); + } } if (removeOnDiskData) { - final File persistDir = computePersistDir(identifier); - long persistFilesSize = 0; - for (FireHydrant hydrant : sink) { - final File innerDir = new File(persistDir, String.valueOf(hydrant.getCount())); - if (innerDir.exists()) { - persistFilesSize += FileUtils.sizeOfDirectory(innerDir); - } - } - removeDirectory(persistDir); - persistedBytes -= persistFilesSize; + removeDirectory(computePersistDir(identifier)); } return null; @@ -1019,7 +1028,6 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdentifier identif new File(persistDir, String.valueOf(indexToPersist.getCount())), indexSpec ); - persistedBytes += FileUtils.sizeOfDirectory(persistedFile); indexToPersist.swapSegment( new QueryableIndexSegment( diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java index 4f94c4f3c3f1..411d7500f930 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java @@ -31,16 +31,16 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.metamx.emitter.EmittingLogger; - import io.druid.common.guava.ThreadRenamingCallable; import io.druid.concurrent.Execs; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; -import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.ScheduledExecutors; +import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; @@ -54,7 +54,6 @@ import io.druid.segment.realtime.plumber.VersioningPolicy; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; - import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; @@ -169,9 +168,9 @@ public QueryRunner getQueryRunner(final Query query) return new QueryRunner() { @Override - public Sequence run(final Query query, final Map responseContext) + public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - return query.run(appenderator, responseContext); + return queryPlus.run(appenderator, responseContext); } }; } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java index 41b1685118c0..18a0fb68f5da 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java @@ -114,12 +114,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio { // Do nothing } - - @Override - public boolean isAnnounced(DataSegment segment) - { - return false; - } }, null, null, diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java deleted file mode 100644 index c7f2200f955b..000000000000 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java +++ /dev/null @@ -1,667 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.appenderator; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.data.input.Committer; -import io.druid.data.input.InputRow; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.Pair; -import io.druid.java.util.common.logger.Logger; -import io.druid.query.SegmentDescriptor; -import io.druid.segment.realtime.FireDepartmentMetrics; -import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; -import io.druid.timeline.DataSegment; -import org.joda.time.DateTime; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Random; -import java.util.Set; -import java.util.TreeMap; -import java.util.stream.Collectors; - -/** - * A FiniteAppenderatorDriver drives an Appenderator to index a finite stream of data. This class does not help you - * index unbounded streams. All handoff is done at the end of indexing. - * - * This class helps with doing things that Appenderators don't, including deciding which segments to use (with a - * SegmentAllocator), publishing segments to the metadata store (with a SegmentPublisher), and monitoring handoff (with - * a SegmentHandoffNotifier). - * - * Note that the commit metadata stored by this class via the underlying Appenderator is not the same metadata as - * you pass in. It's wrapped in some extra metadata needed by the driver. - */ -public class FiniteAppenderatorDriver implements Closeable -{ - private static final Logger log = new Logger(FiniteAppenderatorDriver.class); - - private final Appenderator appenderator; - private final SegmentAllocator segmentAllocator; - private final SegmentHandoffNotifier handoffNotifier; - private final UsedSegmentChecker usedSegmentChecker; - private final ObjectMapper objectMapper; - private final int maxRowsPerSegment; - private final long handoffConditionTimeout; - private final FireDepartmentMetrics metrics; - - private final long maxPersistedSegmentsBytes; - - // All access to "activeSegments" and "lastSegmentId" must be synchronized on "activeSegments". - - // sequenceName -> start of segment interval -> segment we're currently adding data to - private final Map> activeSegments = new TreeMap<>(); - - // sequenceName -> most recently allocated segment - private final Map lastSegmentIds = Maps.newHashMap(); - - // Notified when segments are dropped. - private final Object handoffMonitor = new Object(); - - /** - * Create a driver. - * - * @param appenderator appenderator - * @param segmentAllocator segment allocator - * @param handoffNotifierFactory handoff notifier factory - * @param usedSegmentChecker used segment checker - * @param objectMapper object mapper, used for serde of commit metadata - * @param maxRowsPerSegment maximum number of rows allowed in an entire segment (not a single persist) - * @param handoffConditionTimeout maximum number of millis allowed for handoff (not counting push/publish), zero - * means wait forever. - * @param metrics Firedepartment metrics - */ - public FiniteAppenderatorDriver( - Appenderator appenderator, - SegmentAllocator segmentAllocator, - SegmentHandoffNotifierFactory handoffNotifierFactory, - UsedSegmentChecker usedSegmentChecker, - ObjectMapper objectMapper, - int maxRowsPerSegment, - long maxPersistedSegmentsBytes, - long handoffConditionTimeout, - FireDepartmentMetrics metrics - ) - { - this.appenderator = Preconditions.checkNotNull(appenderator, "appenderator"); - this.segmentAllocator = Preconditions.checkNotNull(segmentAllocator, "segmentAllocator"); - this.handoffNotifier = Preconditions.checkNotNull(handoffNotifierFactory, "handoffNotifierFactory") - .createSegmentHandoffNotifier(appenderator.getDataSource()); - this.usedSegmentChecker = Preconditions.checkNotNull(usedSegmentChecker, "usedSegmentChecker"); - this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); - this.maxRowsPerSegment = maxRowsPerSegment; - this.maxPersistedSegmentsBytes = maxPersistedSegmentsBytes; - this.handoffConditionTimeout = handoffConditionTimeout; - this.metrics = Preconditions.checkNotNull(metrics, "metrics"); - } - - /** - * Perform any initial setup and return currently persisted commit metadata. - * - * Note that this method returns the same metadata you've passed in with your Committers, even though this class - * stores extra metadata on disk. - * - * @return currently persisted commit metadata - */ - public Object startJob() - { - handoffNotifier.start(); - - final FiniteAppenderatorDriverMetadata metadata = objectMapper.convertValue( - appenderator.startJob(), - FiniteAppenderatorDriverMetadata.class - ); - - log.info("Restored metadata[%s].", metadata); - - if (metadata != null) { - synchronized (activeSegments) { - for (Map.Entry> entry : metadata.getActiveSegments().entrySet()) { - final String sequenceName = entry.getKey(); - final TreeMap segmentMap = Maps.newTreeMap(); - - lastSegmentIds.put(sequenceName, metadata.getLastSegmentIds().get(sequenceName)); - activeSegments.put(sequenceName, segmentMap); - - for (SegmentIdentifier identifier : entry.getValue()) { - segmentMap.put(identifier.getInterval().getStartMillis(), identifier); - } - } - } - - return metadata.getCallerMetadata(); - } else { - return null; - } - } - - /** - * Clears out all our state and also calls {@link Appenderator#clear()} on the underlying Appenderator. - */ - public void clear() throws InterruptedException - { - synchronized (activeSegments) { - activeSegments.clear(); - } - appenderator.clear(); - } - - /** - * Add a row. This method may internally incur persisting data added so far. Also, if too large data are persisted, - * it can incur publishing data. This method must not be called concurrently from multiple threads. - * - * @param row the row to add - * @param sequenceName sequenceName for this row's segment - * @param committerSupplier supplier of a committer associated with all data that has been added, including this row - * @param publisher a publisher to publish segments - * @param publish enable publishing or not - * - * @return a pair of a segmentIdentifier and a list of segmentIdentifiers. The lhs of the result pair represents - * segment to which this row was added, or null if segment allocator returned null for this row. The rhs of the result - * pair is the list identifiers of published segments - * - * @throws IOException if there is an I/O error while allocating or writing to a segment - */ - public Pair> add( - final InputRow row, - final String sequenceName, - final Supplier committerSupplier, - final TransactionalSegmentPublisher publisher, - final boolean publish - ) throws IOException, InterruptedException - { - Preconditions.checkNotNull(row, "row"); - Preconditions.checkNotNull(sequenceName, "sequenceName"); - Preconditions.checkNotNull(committerSupplier, "committerSupplier"); - Preconditions.checkNotNull(publisher, "publisher"); - - final SegmentIdentifier identifier = getSegment(row.getTimestamp(), row, sequenceName); - - final List movedOutSegments = new ArrayList<>(); - if (identifier != null) { - try { - final int numRows = appenderator.add(identifier, row, wrapCommitterSupplier(committerSupplier)); - - if (publish && appenderator.getPersistedBytes() > maxPersistedSegmentsBytes) { - // publish segments generated so far - final SegmentsAndMetadata published = publishAndWaitHandoff(publisher, wrapCommitter(committerSupplier.get())); - - // find published activeSegments and move them out - final Map> sequenceToSegmentId = new HashMap<>(); - published.getSegments().forEach(segment -> { - final SegmentIdentifier moveOutSegmentId = SegmentIdentifier.fromDataSegment(segment); - final String moveOutSequence = Appenderators.getSequenceName( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec() - ); - - sequenceToSegmentId.computeIfAbsent(moveOutSequence, key -> new ArrayList<>()).add(moveOutSegmentId); - }); - // Some of the published segments may be already moved out from activeSegments due to the maxRowsPerSegment limit. - // So, here the absent segments are simply ignored. - sequenceToSegmentId.forEach((key, val) -> moveSegmentOut(key, val, true)); - movedOutSegments.addAll( - published.getSegments().stream() - .map(SegmentIdentifier::fromDataSegment) - .collect(Collectors.toList()) - ); - } else if (numRows >= maxRowsPerSegment) { - moveSegmentOut(sequenceName, ImmutableList.of(identifier), false); - movedOutSegments.add(identifier); - } - } - catch (SegmentNotWritableException e) { - throw new ISE(e, "WTF?! Segment[%s] not writable when it should have been.", identifier); - } - } - - return new Pair<>(identifier, movedOutSegments); - } - - /** - * Persist all data indexed through this driver so far. Blocks until complete. - * - * Should be called after all data has been added through {@link #add(InputRow, String, Supplier, TransactionalSegmentPublisher, boolean)} )}. - * - * @param committer committer representing all data that has been added so far - * - * @return commitMetadata persisted - */ - public Object persist(final Committer committer) throws InterruptedException - { - try { - log.info("Persisting data."); - final long start = System.currentTimeMillis(); - final Object commitMetadata = appenderator.persistAll(wrapCommitter(committer)).get(); - log.info("Persisted pending data in %,dms.", System.currentTimeMillis() - start); - return commitMetadata; - } - catch (InterruptedException e) { - throw e; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - /** - * Publish all data indexed through this driver so far, and waits for it to be handed off. Blocks until complete. - * Retries forever on transient failures, but may exit early on permanent failures. - * - * Should be called after all data has been added and persisted through {@link #add(InputRow, String, Supplier, TransactionalSegmentPublisher, boolean)} - * and {@link #persist(Committer)}. - * - * @param publisher publisher to use for this set of segments - * @param committer committer representing all data that has been added so far - * - * @return segments and metadata published if successful, or null if segments could not be handed off due to - * transaction failure with commit metadata. - */ - public SegmentsAndMetadata publishAndWaitHandoff( - final TransactionalSegmentPublisher publisher, - final Committer committer - ) throws InterruptedException - { - final SegmentsAndMetadata segmentsAndMetadata = publishAll(publisher, wrapCommitter(committer)); - - if (segmentsAndMetadata != null) { - final long giveUpAt = handoffConditionTimeout > 0 - ? System.currentTimeMillis() + handoffConditionTimeout - : 0; - - log.info("Awaiting handoff of segments: [%s]", Joiner.on(", ").join(appenderator.getSegments())); - - synchronized (handoffMonitor) { - while (!appenderator.getSegments().isEmpty()) { - - if (giveUpAt == 0) { - handoffMonitor.wait(); - } else { - final long remaining = giveUpAt - System.currentTimeMillis(); - if (remaining > 0) { - handoffMonitor.wait(remaining); - } else { - throw new ISE( - "Segment handoff wait timeout. Segments not yet handed off: [%s]", - Joiner.on(", ").join(appenderator.getSegments()) - ); - } - } - } - } - - log.info("All segments handed off."); - - return new SegmentsAndMetadata( - segmentsAndMetadata.getSegments(), - ((FiniteAppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() - ); - } else { - return null; - } - } - - /** - * Closes this driver. Does not close the underlying Appenderator; you should do that yourself. - */ - @Override - public void close() - { - handoffNotifier.close(); - } - - private SegmentIdentifier getActiveSegment(final DateTime timestamp, final String sequenceName) - { - synchronized (activeSegments) { - final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); - - if (activeSegmentsForSequence == null) { - return null; - } - - final Map.Entry candidateEntry = activeSegmentsForSequence.floorEntry(timestamp.getMillis()); - if (candidateEntry != null && candidateEntry.getValue().getInterval().contains(timestamp)) { - return candidateEntry.getValue(); - } else { - return null; - } - } - } - - /** - * Return a segment usable for "timestamp". May return null if no segment can be allocated. - * - * @param timestamp data timestamp - * @param sequenceName sequenceName for potential segment allocation - * - * @return identifier, or null - * - * @throws IOException if an exception occurs while allocating a segment - */ - private SegmentIdentifier getSegment( - final DateTime timestamp, - final InputRow row, - final String sequenceName - ) throws IOException - { - synchronized (activeSegments) { - final SegmentIdentifier existing = getActiveSegment(timestamp, sequenceName); - if (existing != null) { - return existing; - } else { - // Allocate new segment. - final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); - final SegmentIdentifier newSegment = segmentAllocator.allocate( - timestamp, - row, - sequenceName, - lastSegmentIds.get(sequenceName) - ); - - if (newSegment != null) { - final Long key = newSegment.getInterval().getStartMillis(); - - // If an interval is partitioned with the HashBasedNumberedShardSpec, that partition spec is fixed and cannot - // be modified. It means, segmentAllocator cannot create a new segmentIdentifier and may return an existing - // one. To avoid this, activeSegments are never moved out by setting maxRowsPerSegment to Integer.MAX_VALUE - // and disallowing segment publish in the middle of data ingestion. See IndexTask.newDriver() and - // IndexTask.determineShardSpecs(). - for (SegmentIdentifier identifier : appenderator.getSegments()) { - if (identifier.equals(newSegment)) { - throw new ISE( - "WTF?! Allocated segment[%s] which conflicts with existing segment[%s].", - newSegment, - identifier - ); - } - } - - log.info("New segment[%s] for sequenceName[%s].", newSegment, sequenceName); - if (activeSegmentsForSequence == null) { - activeSegments.put(sequenceName, Maps.newTreeMap()); - } - activeSegments.get(sequenceName).put(key, newSegment); - lastSegmentIds.put(sequenceName, newSegment.getIdentifierAsString()); - } else { - // Well, we tried. - log.warn("Cannot allocate segment for timestamp[%s], sequenceName[%s]. ", timestamp, sequenceName); - } - - return newSegment; - } - } - } - - /** - * Move a set of identifiers out from "active", making way for newer segments. - */ - private boolean moveSegmentOut( - final String sequenceName, - final List identifiers, - boolean ignoreAbsent - ) - { - synchronized (activeSegments) { - final NavigableMap activeSegmentsForSequence = activeSegments.get(sequenceName); - if (activeSegmentsForSequence == null) { - if (ignoreAbsent) { - return false; - } else { - throw new ISE("WTF?! Asked to remove segments for sequenceName[%s] which doesn't exist...", sequenceName); - } - } - - for (final SegmentIdentifier identifier : identifiers) { - log.info("Moving segment[%s] out of active list.", identifier); - final long key = identifier.getInterval().getStartMillis(); - if (!activeSegmentsForSequence.remove(key).equals(identifier)) { - throw new ISE("WTF?! Asked to remove segment[%s] that didn't exist...", identifier); - } - } - return true; - } - } - - /** - * Push and publish all segments to the metadata store. - * - * @param publisher segment publisher - * @param wrappedCommitter wrapped committer (from wrapCommitter) - * - * @return published segments and metadata, or null if segments could not be published due to transaction failure - * with commit metadata. - */ - private SegmentsAndMetadata publishAll( - final TransactionalSegmentPublisher publisher, - final Committer wrappedCommitter - ) throws InterruptedException - { - final List theSegments = ImmutableList.copyOf(appenderator.getSegments()); - - long nTry = 0; - while (true) { - try { - log.info("Pushing segments: [%s]", Joiner.on(", ").join(theSegments)); - final SegmentsAndMetadata segmentsAndMetadata = appenderator.push(theSegments, wrappedCommitter).get(); - - // Sanity check - if (!segmentsToIdentifiers(segmentsAndMetadata.getSegments()).equals(Sets.newHashSet(theSegments))) { - throw new ISE( - "WTF?! Pushed different segments than requested. Pushed[%s], requested[%s].", - Joiner.on(", ").join(identifiersToStrings(segmentsToIdentifiers(segmentsAndMetadata.getSegments()))), - Joiner.on(", ").join(identifiersToStrings(theSegments)) - ); - } - - log.info( - "Publishing segments with commitMetadata[%s]: [%s]", - segmentsAndMetadata.getCommitMetadata(), - Joiner.on(", ").join(segmentsAndMetadata.getSegments()) - ); - - if (segmentsAndMetadata.getSegments().isEmpty()) { - log.info("Nothing to publish, skipping publish step."); - } else { - final boolean published = publisher.publishSegments( - ImmutableSet.copyOf(segmentsAndMetadata.getSegments()), - ((FiniteAppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() - ); - - if (published) { - log.info("Published segments, awaiting handoff."); - } else { - log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); - if (usedSegmentChecker.findUsedSegments(segmentsToIdentifiers(segmentsAndMetadata.getSegments())) - .equals(Sets.newHashSet(segmentsAndMetadata.getSegments()))) { - log.info("Our segments really do exist, awaiting handoff."); - } else { - log.warn("Our segments don't exist, giving up."); - return null; - } - } - } - - for (final DataSegment dataSegment : segmentsAndMetadata.getSegments()) { - handoffNotifier.registerSegmentHandoffCallback( - new SegmentDescriptor( - dataSegment.getInterval(), - dataSegment.getVersion(), - dataSegment.getShardSpec().getPartitionNum() - ), - MoreExecutors.sameThreadExecutor(), - new Runnable() - { - @Override - public void run() - { - final SegmentIdentifier identifier = SegmentIdentifier.fromDataSegment(dataSegment); - log.info("Segment[%s] successfully handed off, dropping.", identifier); - metrics.incrementHandOffCount(); - final ListenableFuture dropFuture = appenderator.drop(identifier); - Futures.addCallback( - dropFuture, - new FutureCallback() - { - @Override - public void onSuccess(Object result) - { - synchronized (handoffMonitor) { - handoffMonitor.notifyAll(); - } - } - - @Override - public void onFailure(Throwable e) - { - log.warn(e, "Failed to drop segment[%s]?!"); - synchronized (handoffMonitor) { - handoffMonitor.notifyAll(); - } - } - } - ); - } - } - ); - } - - return segmentsAndMetadata; - } - catch (InterruptedException e) { - throw e; - } - catch (Exception e) { - final long sleepMillis = computeNextRetrySleep(++nTry); - log.warn(e, "Failed publishAll (try %d), retrying in %,dms.", nTry, sleepMillis); - Thread.sleep(sleepMillis); - } - } - } - - private Supplier wrapCommitterSupplier(final Supplier committerSupplier) - { - return new Supplier() - { - @Override - public Committer get() - { - return wrapCommitter(committerSupplier.get()); - } - }; - } - - private Committer wrapCommitter(final Committer committer) - { - synchronized (activeSegments) { - final FiniteAppenderatorDriverMetadata wrappedMetadata = new FiniteAppenderatorDriverMetadata( - ImmutableMap.copyOf( - Maps.transformValues( - activeSegments, - new Function, List>() - { - @Override - public List apply(NavigableMap input) - { - return ImmutableList.copyOf(input.values()); - } - } - ) - ), - ImmutableMap.copyOf(lastSegmentIds), - committer.getMetadata() - ); - - return new Committer() - { - @Override - public Object getMetadata() - { - return wrappedMetadata; - } - - @Override - public void run() - { - committer.run(); - } - }; - } - } - - private static long computeNextRetrySleep(final long nTry) - { - final long baseSleepMillis = 1000; - final long maxSleepMillis = 60000; - final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * new Random().nextGaussian(), 0), 2); - return (long) (Math.min(maxSleepMillis, baseSleepMillis * Math.pow(2, nTry)) * fuzzyMultiplier); - } - - private static Set segmentsToIdentifiers(final Iterable segments) - { - return FluentIterable.from(segments) - .transform( - new Function() - { - @Override - public SegmentIdentifier apply(DataSegment segment) - { - return SegmentIdentifier.fromDataSegment(segment); - } - } - ).toSet(); - } - - private static Iterable identifiersToStrings(final Iterable identifiers) - { - return FluentIterable.from(identifiers) - .transform( - new Function() - { - @Override - public String apply(SegmentIdentifier input) - { - return input.getIdentifierAsString(); - } - } - ); - } -} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java index 759be21a3074..15b2934f72fa 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java @@ -20,7 +20,6 @@ package io.druid.segment.realtime.appenderator; import io.druid.data.input.InputRow; -import org.joda.time.DateTime; import java.io.IOException; @@ -29,14 +28,13 @@ public interface SegmentAllocator /** * Allocates a new segment for a given timestamp. * - * @param timestamp timestamp of the event which triggered this allocation request + * @param row the event which triggered this allocation request * @param sequenceName sequenceName for this allocation * @param previousSegmentId segment identifier returned on the previous call to allocate for your sequenceName * * @return the pending segment identifier, or null if it was impossible to allocate a new segment */ SegmentIdentifier allocate( - DateTime timestamp, InputRow row, String sequenceName, String previousSegmentId diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 37ef7489831f..0caa56316199 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -275,7 +275,7 @@ public QueryRunner apply(final FireHydrant hydrant) */ private QueryRunner withPerSinkMetrics( final QueryRunner sinkRunner, - final QueryToolChest> queryToolChest, + final QueryToolChest> queryToolChest, final String sinkSegmentIdentifier, final AtomicLong cpuTimeAccumulator ) diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java index ef9a8ad03fee..9bcd7761152d 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java @@ -28,6 +28,7 @@ import io.druid.data.input.impl.InputRowParser; import org.joda.time.Interval; +import java.io.File; import java.io.IOException; /** @@ -61,10 +62,10 @@ public Interval getInterval() } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { return new PredicateFirehose( - delegate.connect(parser), + delegate.connect(parser, temporaryDirectory), new Predicate() { @Override diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java index 536c1edf88c7..4dc3c9627528 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java @@ -29,6 +29,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import java.io.File; import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -52,9 +53,9 @@ public CombiningFirehoseFactory( } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { - return new CombiningFirehose(parser); + return new CombiningFirehose(parser, temporaryDirectory); } @JsonProperty("delegates") @@ -63,16 +64,18 @@ public List getDelegateFactoryList() return delegateFactoryList; } - public class CombiningFirehose implements Firehose + class CombiningFirehose implements Firehose { private final InputRowParser parser; + private final File temporaryDirectory; private final Iterator firehoseFactoryIterator; private volatile Firehose currentFirehose; - public CombiningFirehose(InputRowParser parser) throws IOException + CombiningFirehose(InputRowParser parser, File temporaryDirectory) throws IOException { this.firehoseFactoryIterator = delegateFactoryList.iterator(); this.parser = parser; + this.temporaryDirectory = temporaryDirectory; nextFirehose(); } @@ -84,7 +87,7 @@ private void nextFirehose() currentFirehose.close(); } - currentFirehose = firehoseFactoryIterator.next().connect(parser); + currentFirehose = firehoseFactoryIterator.next().connect(parser, temporaryDirectory); } catch (IOException e) { if (currentFirehose != null) { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index 3f20139fd8ec..1cf9911c7c26 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -54,6 +54,7 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.util.Collection; @@ -103,7 +104,7 @@ public EventReceiverFirehoseFactory( } @Override - public Firehose connect(MapInputRowParser firehoseParser) throws IOException + public Firehose connect(MapInputRowParser firehoseParser, File temporaryDirectory) throws IOException { log.info("Connecting firehose: %s", serviceName); final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser); diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java index 300b6d139229..fbf498ce5bac 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java @@ -27,6 +27,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import java.io.File; import java.io.IOException; /** @@ -60,12 +61,12 @@ public int getCount() } @Override - public Firehose connect(final InputRowParser parser) throws IOException + public Firehose connect(final InputRowParser parser, File temporaryDirectory) throws IOException { return new Firehose() { private int i = 0; - private Firehose delegateFirehose = delegate.connect(parser); + private Firehose delegateFirehose = delegate.connect(parser, temporaryDirectory); @Override public boolean hasMore() diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java new file mode 100644 index 000000000000..4a604ae7f93b --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java @@ -0,0 +1,74 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.firehose; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.input.impl.PrefetchableTextFilesFirehoseFactory; +import io.druid.java.util.common.CompressionUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.util.Collection; +import java.util.List; + +public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory +{ + private final List uris; + + @JsonCreator + public HttpFirehoseFactory( + @JsonProperty("uris") List uris, + @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, + @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, + @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, + @JsonProperty("fetchTimeout") Long fetchTimeout, + @JsonProperty("maxFetchRetry") Integer maxFetchRetry + ) + { + super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); + this.uris = uris; + } + + @JsonProperty + public List getUris() + { + return uris; + } + + @Override + protected Collection initObjects() + { + return uris; + } + + @Override + protected InputStream openObjectStream(URI object) throws IOException + { + return object.toURL().openConnection().getInputStream(); + } + + @Override + protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java index 3da59a8f5fda..099b7c12a3e1 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java @@ -30,15 +30,14 @@ import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg; import com.ircclouds.irc.api.listeners.VariousMessageListenerAdapter; import com.ircclouds.irc.api.state.IIRCState; - import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; - import org.joda.time.DateTime; +import java.io.File; import java.io.IOException; import java.util.List; import java.util.UUID; @@ -101,7 +100,7 @@ public List getChannels() } @Override - public Firehose connect(final IrcInputRowParser firehoseParser) throws IOException + public Firehose connect(final IrcInputRowParser firehoseParser, File temporaryDirectory) throws IOException { final IRCApi irc = new IRCApiImpl(false); final LinkedBlockingQueue> queue = new LinkedBlockingQueue>(); diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index 381fefeed14f..6b80f2e96fbc 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -21,30 +21,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; +import com.google.common.base.Preconditions; import com.metamx.emitter.EmittingLogger; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.impl.FileIteratingFirehose; +import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import io.druid.data.input.impl.StringInputRowParser; -import io.druid.java.util.common.IAE; -import io.druid.java.util.common.ISE; - +import io.druid.java.util.common.CompressionUtils; import org.apache.commons.io.FileUtils; -import org.apache.commons.io.LineIterator; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; /** */ -public class LocalFirehoseFactory implements FirehoseFactory +public class LocalFirehoseFactory extends AbstractTextFilesFirehoseFactory { private static final EmittingLogger log = new EmittingLogger(LocalFirehoseFactory.class); @@ -84,55 +77,26 @@ public StringInputRowParser getParser() } @Override - public Firehose connect(StringInputRowParser firehoseParser) throws IOException + protected Collection initObjects() { - if (baseDir == null) { - throw new IAE("baseDir is null"); - } - log.info("Searching for all [%s] in and beneath [%s]", filter, baseDir.getAbsoluteFile()); - - Collection foundFiles = FileUtils.listFiles( - baseDir.getAbsoluteFile(), + final Collection files = FileUtils.listFiles( + Preconditions.checkNotNull(baseDir).getAbsoluteFile(), new WildcardFileFilter(filter), TrueFileFilter.INSTANCE ); + log.info("Initialized with " + files + " files"); + return files; + } - if (foundFiles == null || foundFiles.isEmpty()) { - throw new ISE("Found no files to ingest! Check your schema."); - } - log.info ("Found files: " + foundFiles); - - final LinkedList files = Lists.newLinkedList( - foundFiles - ); - - return new FileIteratingFirehose( - new Iterator() - { - @Override - public boolean hasNext() - { - return !files.isEmpty(); - } - - @Override - public LineIterator next() - { - try { - return FileUtils.lineIterator(files.poll()); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } + @Override + protected InputStream openObjectStream(File object) throws IOException + { + return FileUtils.openInputStream(object); + } - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }, - firehoseParser - ); + @Override + protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException + { + return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java deleted file mode 100644 index 53a586ce4f28..000000000000 --- a/server/src/main/java/io/druid/segment/realtime/firehose/ReplayableFirehoseFactory.java +++ /dev/null @@ -1,319 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.fasterxml.jackson.dataformat.smile.SmileGenerator; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Iterators; -import com.google.common.collect.Ordering; -import com.google.common.io.CountingOutputStream; -import com.google.common.io.Files; -import com.metamx.emitter.EmittingLogger; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.InputRow; -import io.druid.data.input.Row; -import io.druid.data.input.Rows; -import io.druid.data.input.impl.InputRowParser; -import io.druid.guice.annotations.Smile; -import io.druid.java.util.common.parsers.ParseException; -import io.druid.utils.Runnables; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -/** - * Creates a wrapper firehose that writes from another firehose to disk and then serves nextRow() from disk. Useful for - * tasks that require multiple passes through the data to prevent multiple remote fetches. Also has support for - * retrying fetches if the underlying firehose throws an exception while the local cache is being generated. - */ -public class ReplayableFirehoseFactory implements FirehoseFactory -{ - private static final EmittingLogger log = new EmittingLogger(ReplayableFirehoseFactory.class); - private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; - private static final int DEFAULT_MAX_TEMP_FILE_SIZE = 250000000; - private static final int DEFAULT_READ_FIREHOSE_RETRIES = 3; - - private final FirehoseFactory delegateFactory; - private final boolean reportParseExceptions; - - // This is *roughly* the max size of the temp files that will be generated, but they may be slightly larger. The - // reason for the approximation is that we're not forcing flushes after writing to the generator so the number of - // bytes written to the stream won't be updated until the flush occurs. It's probably more important to optimize for - // I/O speed rather than maintaining a strict max on the size of the temp file before it's split. - private final int maxTempFileSize; - - private final int readFirehoseRetries; - private final ObjectMapper smileMapper; - - private ReplayableFirehose firehose; - - @JsonCreator - public ReplayableFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegateFactory, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("maxTempFileSize") Integer maxTempFileSize, - @JsonProperty("readFirehoseRetries") Integer readFirehoseRetries, - @Smile @JacksonInject ObjectMapper smileMapper - ) - { - Preconditions.checkNotNull(delegateFactory, "delegate cannot be null"); - Preconditions.checkArgument( - !(delegateFactory instanceof ReplayableFirehoseFactory), - "Refusing to wrap another ReplayableFirehoseFactory" - ); - - this.delegateFactory = delegateFactory; - this.reportParseExceptions = reportParseExceptions == null - ? DEFAULT_REPORT_PARSE_EXCEPTIONS - : reportParseExceptions; - this.maxTempFileSize = maxTempFileSize == null ? DEFAULT_MAX_TEMP_FILE_SIZE : maxTempFileSize; - this.readFirehoseRetries = readFirehoseRetries == null ? DEFAULT_READ_FIREHOSE_RETRIES : readFirehoseRetries; - - this.smileMapper = smileMapper; - - log.info(this.toString()); - } - - @Override - public Firehose connect(InputRowParser parser) throws IOException - { - if (firehose == null) { - firehose = new ReplayableFirehose(parser); - } else { - log.info("Rewinding and returning existing firehose"); - firehose.rewind(); - } - - return firehose; - } - - public class ReplayableFirehose implements Firehose - { - private final List files = new ArrayList<>(); - private final List dimensions; - - private int fileIndex = 0; - private JsonFactory jsonFactory; - private JsonParser jsonParser; - private Iterator it; - - public ReplayableFirehose(InputRowParser parser) throws IOException - { - jsonFactory = smileMapper.getFactory(); - - if (jsonFactory instanceof SmileFactory) { - jsonFactory = ((SmileFactory) jsonFactory).enable(SmileGenerator.Feature.CHECK_SHARED_STRING_VALUES); - } - - long counter = 0, totalBytes = 0, unparseable = 0, retryCount = 0; - Set dimensionScratch = new HashSet<>(); - - File tmpDir = Files.createTempDir(); - tmpDir.deleteOnExit(); - - long startTime = System.nanoTime(); - boolean isDone = false; - do { - deleteTempFiles(); - try (Firehose delegateFirehose = delegateFactory.connect(parser)) { - while (delegateFirehose.hasMore()) { - File tmpFile = File.createTempFile("replayable-", null, tmpDir); - tmpFile.deleteOnExit(); - - files.add(tmpFile); - log.debug("Created file [%s]", tmpFile.getAbsolutePath()); - - try (CountingOutputStream cos = new CountingOutputStream(new FileOutputStream(tmpFile)); - JsonGenerator generator = jsonFactory.createGenerator(cos)) { - - while (delegateFirehose.hasMore() && cos.getCount() < getMaxTempFileSize()) { - try { - InputRow row = delegateFirehose.nextRow(); - generator.writeObject(row); - dimensionScratch.addAll(row.getDimensions()); - counter++; - } - catch (ParseException e) { - if (reportParseExceptions) { - throw e; - } - unparseable++; - } - } - - totalBytes += cos.getCount(); - } - } - isDone = true; - } - catch (Exception e) { - if (++retryCount <= readFirehoseRetries && !(e instanceof ParseException)) { - log.error(e, "Delegate firehose threw an exception, retrying (%d of %d)", retryCount, readFirehoseRetries); - } else { - log.error(e, "Delegate firehose threw an exception, retries exhausted, aborting"); - Throwables.propagate(e); - } - } - } while (!isDone); - - log.info( - "Finished reading from firehose in [%,dms], [%,d] events parsed, [%,d] bytes written, [%,d] events unparseable", - (System.nanoTime() - startTime) / 1000000, - counter, - totalBytes, - unparseable - ); - - dimensions = Ordering.natural().immutableSortedCopy(dimensionScratch); - - if (counter == 0) { - log.warn("Firehose contains no events!"); - deleteTempFiles(); - it = Iterators.emptyIterator(); - } else { - jsonParser = jsonFactory.createParser(files.get(fileIndex)); - it = jsonParser.readValuesAs(Row.class); - } - } - - @Override - public boolean hasMore() - { - if (it.hasNext()) { - return true; - } - - try { - if (jsonParser != null) { - jsonParser.close(); - } - - if (++fileIndex >= files.size() || files.get(fileIndex).length() == 0) { - return false; - } - - jsonParser = jsonFactory.createParser(files.get(fileIndex)); - it = jsonParser.readValuesAs(Row.class); - return true; - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public InputRow nextRow() - { - return Rows.toCaseInsensitiveInputRow(it.next(), dimensions); - } - - @Override - public Runnable commit() - { - return Runnables.getNoopRunnable(); - } - - /** - * Closes the firehose by closing the input stream and setting an empty iterator. The underlying cache files - * backing the firehose are retained for when the firehose is "replayed" by calling rewind(). The cache files are - * deleted by File.deleteOnExit() when the process exits. - */ - @Override - public void close() throws IOException - { - if (jsonParser != null) { - jsonParser.close(); - } - it = Iterators.emptyIterator(); - } - - private void rewind() throws IOException - { - close(); - - if (!files.isEmpty()) { - fileIndex = 0; - jsonParser = jsonFactory.createParser(files.get(fileIndex)); - it = jsonParser.readValuesAs(Row.class); - } - } - - private void deleteTempFiles() - { - for (File file : files) { - log.debug("Deleting temp file: %s", file.getAbsolutePath()); - file.delete(); - } - - files.clear(); - } - } - - @JsonProperty("delegate") - public FirehoseFactory getDelegateFactory() - { - return delegateFactory; - } - - @JsonProperty("reportParseExceptions") - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @JsonProperty("maxTempFileSize") - public int getMaxTempFileSize() - { - return maxTempFileSize; - } - - @JsonProperty("readFirehoseRetries") - public int getReadFirehoseRetries() - { - return readFirehoseRetries; - } - - @Override - public String toString() - { - return "ReplayableFirehoseFactory{" + - "delegateFactory=" + delegateFactory + - ", reportParseExceptions=" + reportParseExceptions + - ", maxTempFileSize=" + maxTempFileSize + - ", readFirehoseRetries=" + readFirehoseRetries + - '}'; - } -} diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java index 8b64ae1d3381..5ac8a57cbb93 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java @@ -29,6 +29,7 @@ import io.druid.data.input.impl.InputRowParser; import org.joda.time.DateTime; +import java.io.File; import java.io.IOException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -53,21 +54,21 @@ public TimedShutoffFirehoseFactory( } @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { - return new TimedShutoffFirehose(parser); + return new TimedShutoffFirehose(parser, temporaryDirectory); } - public class TimedShutoffFirehose implements Firehose + class TimedShutoffFirehose implements Firehose { private final Firehose firehose; private final ScheduledExecutorService exec; private final Object shutdownLock = new Object(); private volatile boolean shutdown = false; - public TimedShutoffFirehose(InputRowParser parser) throws IOException + TimedShutoffFirehose(InputRowParser parser, File temporaryDirectory) throws IOException { - firehose = delegateFactory.connect(parser); + firehose = delegateFactory.connect(parser, temporaryDirectory); exec = Execs.scheduledSingleThreaded("timed-shutoff-firehose-%d"); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java b/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java index f77c3d92ffa3..b2bdead3ef2f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java @@ -148,7 +148,7 @@ static boolean isHandOffComplete(List serverView, Segm @Override public boolean apply(DruidServerMetadata input) { - return input.isAssignable(); + return input.segmentReplicatable(); } } )) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 161cc6a3905a..1e1b3cb18092 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -42,10 +42,10 @@ import io.druid.concurrent.TaskThreadPriority; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; -import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.concurrent.ScheduledExecutors; +import io.druid.java.util.common.granularity.Granularity; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -424,12 +424,11 @@ public void doRun() metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime); metrics.incrementMergeTimeMillis(mergeStopwatch.elapsed(TimeUnit.MILLISECONDS)); - QueryableIndex index = indexIO.loadIndex(mergedFile); log.info("Pushing [%s] to deep storage", sink.getSegment().getIdentifier()); DataSegment segment = dataSegmentPusher.push( mergedFile, - sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) + sink.getSegment().withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes)) ); log.info("Inserting [%s] to the metadata store", sink.getSegment().getIdentifier()); segmentPublisher.publishSegment(segment); @@ -861,6 +860,7 @@ protected void abandonSegment(final long truncatedTime, final Sink sink) ); for (FireHydrant hydrant : sink) { cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); + hydrant.getSegment().close(); } synchronized (handoffCondition) { handoffCondition.notifyAll(); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java b/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java index 6c76786fe020..d10b8c22864c 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java @@ -48,6 +48,7 @@ boolean registerSegmentHandoffCallback( /** * Perform any final processing and clean up after ourselves. */ + @Override void close(); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index ddebc567c282..bd3dc8e8518e 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -228,7 +228,7 @@ public String apply(@Nullable AggregatorFactory input) public int getNumRows() { synchronized (hydrantLock) { - return numRowsExcludingCurrIndex.get() + currHydrant.getIndex().size(); + return numRowsExcludingCurrIndex.get() + getNumRowsInMemory(); } } diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java index 49252c8c0ad6..791e2cf2ecd7 100644 --- a/server/src/main/java/io/druid/server/QueryManager.java +++ b/server/src/main/java/io/druid/server/QueryManager.java @@ -57,6 +57,7 @@ public boolean cancelQuery(String id) return success; } + @Override public void registerQuery(Query query, final ListenableFuture future) { final String id = query.getId(); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index f161e9a064bf..2ff2766b6c39 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -25,11 +25,11 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.MapMaker; import com.google.common.io.CountingOutputStream; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.DirectDruidClient; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.ISE; @@ -40,9 +40,9 @@ import io.druid.query.DruidMetrics; import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; -import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -185,13 +185,19 @@ public Response doPost( final String currThreadName = Thread.currentThread().getName(); try { + query = context.getObjectMapper().readValue(in, Query.class); queryId = query.getId(); if (queryId == null) { queryId = UUID.randomUUID().toString(); query = query.withId(queryId); } - query = QueryContexts.withDefaultTimeout(query, config.getDefaultQueryTimeout()); + + query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(query, config); + final Map responseContext = DirectDruidClient.makeResponseContextForQuery( + query, + System.currentTimeMillis() + ); toolChest = warehouse.getToolChest(query); @@ -226,8 +232,7 @@ public Response doPost( ); } - final Map responseContext = new MapMaker().makeMap(); - final Sequence res = query.run(texasRanger, responseContext); + final Sequence res = QueryPlus.wrap(query).run(texasRanger, responseContext); if (prevEtag != null && prevEtag.equals(responseContext.get(HDR_ETAG))) { return Response.notModified().build(); @@ -329,6 +334,9 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE responseContext.remove(HDR_ETAG); } + responseContext.remove(DirectDruidClient.QUERY_FAIL_TIME); + responseContext.remove(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED); + //Limit the response-context header, see https://github.com/druid-io/druid/issues/2331 //Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString() //and encodes the string using ASCII, so 1 char is = 1 byte diff --git a/server/src/main/java/io/druid/server/SegmentManager.java b/server/src/main/java/io/druid/server/SegmentManager.java new file mode 100644 index 000000000000..b27bbe8752f9 --- /dev/null +++ b/server/src/main/java/io/druid/server/SegmentManager.java @@ -0,0 +1,266 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Ordering; +import com.google.inject.Inject; +import com.metamx.emitter.EmittingLogger; +import io.druid.common.guava.SettableSupplier; +import io.druid.segment.ReferenceCountingSegment; +import io.druid.segment.Segment; +import io.druid.segment.loading.SegmentLoader; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.PartitionHolder; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * This class is responsible for managing data sources and their states like timeline, total segment size, and number of + * segments. All public methods of this class must be thread-safe. + */ +public class SegmentManager +{ + private static final EmittingLogger log = new EmittingLogger(SegmentManager.class); + + private final SegmentLoader segmentLoader; + private final ConcurrentHashMap dataSources = new ConcurrentHashMap<>(); + + /** + * Represent the state of a data source including the timeline, total segment size, and number of segments. + */ + public static class DataSourceState + { + private final VersionedIntervalTimeline timeline = + new VersionedIntervalTimeline<>(Ordering.natural()); + private long totalSegmentSize; + private long numSegments; + + private void addSegment(DataSegment segment) + { + totalSegmentSize += segment.getSize(); + numSegments++; + } + + private void removeSegment(DataSegment segment) + { + totalSegmentSize -= segment.getSize(); + numSegments--; + } + + public VersionedIntervalTimeline getTimeline() + { + return timeline; + } + + public long getTotalSegmentSize() + { + return totalSegmentSize; + } + + public long getNumSegments() + { + return numSegments; + } + + public boolean isEmpty() + { + return numSegments == 0; + } + } + + @Inject + public SegmentManager( + SegmentLoader segmentLoader + ) + { + this.segmentLoader = segmentLoader; + } + + @VisibleForTesting + Map getDataSources() + { + return dataSources; + } + + /** + * Returns a map of dataSource to the total byte size of segments managed by this segmentManager. This method should + * be used carefully because the returned map might be different from the actual data source states. + * + * @return a map of dataSources and their total byte sizes + */ + public Map getDataSourceSizes() + { + return dataSources.entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getTotalSegmentSize())); + } + + /** + * Returns a map of dataSource to the number of segments managed by this segmentManager. This method should be + * carefully because the returned map might be different from the actual data source states. + * + * @return a map of dataSources and number of segments + */ + public Map getDataSourceCounts() + { + return dataSources.entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getNumSegments())); + } + + public boolean isSegmentCached(final DataSegment segment) throws SegmentLoadingException + { + return segmentLoader.isSegmentLoaded(segment); + } + + @Nullable + public VersionedIntervalTimeline getTimeline(String dataSource) + { + final DataSourceState dataSourceState = dataSources.get(dataSource); + return dataSourceState == null ? null : dataSourceState.getTimeline(); + } + + /** + * Load a single segment. + * + * @param segment segment to load + * + * @return true if the segment was newly loaded, false if it was already loaded + * + * @throws SegmentLoadingException if the segment cannot be loaded + */ + public boolean loadSegment(final DataSegment segment) throws SegmentLoadingException + { + final Segment adapter = getAdapter(segment); + + final SettableSupplier resultSupplier = new SettableSupplier<>(); + + // compute() is used to ensure that the operation for a data source is executed atomically + dataSources.compute( + segment.getDataSource(), + (k, v) -> { + final DataSourceState dataSourceState = v == null ? new DataSourceState() : v; + final VersionedIntervalTimeline loadedIntervals = + dataSourceState.getTimeline(); + final PartitionHolder entry = loadedIntervals.findEntry( + segment.getInterval(), + segment.getVersion() + ); + + if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { + log.warn("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); + resultSupplier.set(false); + } else { + loadedIntervals.add( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter)) + ); + dataSourceState.addSegment(segment); + resultSupplier.set(true); + } + return dataSourceState; + } + ); + + return resultSupplier.get(); + } + + private Segment getAdapter(final DataSegment segment) throws SegmentLoadingException + { + final Segment adapter; + try { + adapter = segmentLoader.getSegment(segment); + } + catch (SegmentLoadingException e) { + try { + segmentLoader.cleanup(segment); + } + catch (SegmentLoadingException e1) { + e.addSuppressed(e1); + } + throw e; + } + + if (adapter == null) { + throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); + } + return adapter; + } + + public void dropSegment(final DataSegment segment) throws SegmentLoadingException + { + final String dataSource = segment.getDataSource(); + + // compute() is used to ensure that the operation for a data source is executed atomically + dataSources.compute( + dataSource, + (dataSourceName, dataSourceState) -> { + if (dataSourceState == null) { + log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSourceName); + } else { + final VersionedIntervalTimeline loadedIntervals = + dataSourceState.getTimeline(); + + final PartitionChunk removed = loadedIntervals.remove( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk(null) + ); + final ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); + + if (oldQueryable != null) { + dataSourceState.removeSegment(segment); + + try { + log.info("Attempting to close segment %s", segment.getIdentifier()); + oldQueryable.close(); + } + catch (IOException e) { + log.makeAlert(e, "Exception closing segment") + .addData("dataSource", dataSourceName) + .addData("segmentId", segment.getIdentifier()) + .emit(); + } + } else { + log.info( + "Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.", + dataSourceName, + segment.getInterval(), + segment.getVersion() + ); + } + } + + // Returning null removes the entry of dataSource from the map + return dataSourceState == null || dataSourceState.isEmpty() ? null : dataSourceState; + } + ); + + segmentLoader.cleanup(segment); + } +} diff --git a/server/src/main/java/io/druid/server/audit/AuditManagerProvider.java b/server/src/main/java/io/druid/server/audit/AuditManagerProvider.java index 8fbf8c85eae4..cb3d1f2e7f4c 100644 --- a/server/src/main/java/io/druid/server/audit/AuditManagerProvider.java +++ b/server/src/main/java/io/druid/server/audit/AuditManagerProvider.java @@ -24,5 +24,6 @@ public interface AuditManagerProvider extends Provider { + @Override public AuditManager get(); } diff --git a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java index 08e3bca4cbfd..b73322f726e7 100644 --- a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -24,10 +24,12 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; - import io.druid.common.utils.UUIDUtils; import io.druid.curator.announcement.Announcer; import io.druid.java.util.common.ISE; @@ -38,8 +40,11 @@ import org.apache.curator.utils.ZKPaths; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; @@ -47,7 +52,7 @@ /** */ -public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer +public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer { private static final Logger log = new Logger(BatchDataSegmentAnnouncer.class); @@ -64,6 +69,9 @@ public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer private final Map segmentLookup = Maps.newConcurrentMap(); private final Function segmentTransformer; + private final SegmentChangeRequestHistory changes = new SegmentChangeRequestHistory(); + private final SegmentZNode dummyZnode; + @Inject public BatchDataSegmentAnnouncer( DruidServerMetadata server, @@ -73,7 +81,6 @@ public BatchDataSegmentAnnouncer( ObjectMapper jsonMapper ) { - super(server, zkPaths, announcer, jsonMapper); this.config = config; this.announcer = announcer; this.jsonMapper = jsonMapper; @@ -95,18 +102,37 @@ public DataSegment apply(DataSegment input) return rv; } }; + + if (this.config.isSkipSegmentAnnouncementOnZk()) { + dummyZnode = new SegmentZNode("PLACE_HOLDER_ONLY"); + } else { + dummyZnode = null; + } } @Override public void announceSegment(DataSegment segment) throws IOException { - DataSegment toAnnounce = segmentTransformer.apply(segment); - int newBytesLen = jsonMapper.writeValueAsBytes(toAnnounce).length; - if (newBytesLen > config.getMaxBytesPerNode()) { - throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); + if (segmentLookup.containsKey(segment)) { + log.info("Skipping announcement of segment [%s]. Announcement exists already."); + return; } + DataSegment toAnnounce = segmentTransformer.apply(segment); + synchronized (lock) { + changes.addSegmentChangeRequest(new SegmentChangeRequestLoad(toAnnounce)); + + if (config.isSkipSegmentAnnouncementOnZk()) { + segmentLookup.put(segment, dummyZnode); + return; + } + + int newBytesLen = jsonMapper.writeValueAsBytes(toAnnounce).length; + if (newBytesLen > config.getMaxBytesPerNode()) { + throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); + } + boolean done = false; if (!availableZNodes.isEmpty()) { // update existing batch @@ -155,13 +181,20 @@ public void announceSegment(DataSegment segment) throws IOException @Override public void unannounceSegment(DataSegment segment) throws IOException { - final SegmentZNode segmentZNode = segmentLookup.remove(segment); - if (segmentZNode == null) { - log.warn("No path to unannounce segment[%s]", segment.getIdentifier()); - return; - } - synchronized (lock) { + final SegmentZNode segmentZNode = segmentLookup.remove(segment); + + if (segmentZNode == null) { + log.warn("No path to unannounce segment[%s]", segment.getIdentifier()); + return; + } + + changes.addSegmentChangeRequest(new SegmentChangeRequestDrop(segment)); + + if (config.isSkipSegmentAnnouncementOnZk()) { + return; + } + segmentZNode.removeSegment(segment); log.info("Unannouncing segment[%s] at path[%s]", segment.getIdentifier(), segmentZNode.getPath()); @@ -178,38 +211,60 @@ public void unannounceSegment(DataSegment segment) throws IOException @Override public void announceSegments(Iterable segments) throws IOException { - Iterable toAnnounce = Iterables.transform(segments, segmentTransformer); SegmentZNode segmentZNode = new SegmentZNode(makeServedSegmentPath()); Set batch = Sets.newHashSet(); + List changesBatch = new ArrayList<>(); + int byteSize = 0; int count = 0; - for (DataSegment segment : toAnnounce) { - int newBytesLen = jsonMapper.writeValueAsBytes(segment).length; + synchronized (lock) { + for (DataSegment ds : segments) { - if (newBytesLen > config.getMaxBytesPerNode()) { - throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); - } + if (segmentLookup.containsKey(ds)) { + log.info("Skipping announcement of segment [%s]. Announcement exists already."); + return; + } - if (count >= config.getSegmentsPerNode() || byteSize + newBytesLen > config.getMaxBytesPerNode()) { - segmentZNode.addSegments(batch); - announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); + DataSegment segment = segmentTransformer.apply(ds); - segmentZNode = new SegmentZNode(makeServedSegmentPath()); - batch = Sets.newHashSet(); - count = 0; - byteSize = 0; - } + changesBatch.add(new SegmentChangeRequestLoad(segment)); - log.info("Announcing segment[%s] at path[%s]", segment.getIdentifier(), segmentZNode.getPath()); - segmentLookup.put(segment, segmentZNode); - batch.add(segment); - count++; - byteSize += newBytesLen; + if (config.isSkipSegmentAnnouncementOnZk()) { + segmentLookup.put(segment, dummyZnode); + continue; + } + + int newBytesLen = jsonMapper.writeValueAsBytes(segment).length; + + if (newBytesLen > config.getMaxBytesPerNode()) { + throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); + } + + if (count >= config.getSegmentsPerNode() || byteSize + newBytesLen > config.getMaxBytesPerNode()) { + segmentZNode.addSegments(batch); + announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); + + segmentZNode = new SegmentZNode(makeServedSegmentPath()); + batch = Sets.newHashSet(); + count = 0; + byteSize = 0; + } + + log.info("Announcing segment[%s] at path[%s]", segment.getIdentifier(), segmentZNode.getPath()); + segmentLookup.put(segment, segmentZNode); + batch.add(segment); + count++; + byteSize += newBytesLen; + } } - segmentZNode.addSegments(batch); - announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); + changes.addSegmentChangeRequests(changesBatch); + + if (!config.isSkipSegmentAnnouncementOnZk()) { + segmentZNode.addSegments(batch); + announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); + } } @Override @@ -220,21 +275,46 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } } - @Override - public boolean isAnnounced(DataSegment segment) + /** + * Returns Future that lists the segment load/drop requests since given counter. + */ + public ListenableFuture getSegmentChangesSince(SegmentChangeRequestHistory.Counter counter) { - return segmentLookup.containsKey(segment); + if (counter.getCounter() < 0) { + synchronized (lock) { + Iterable segments = Iterables.transform( + segmentLookup.keySet(), + new Function() + { + @Nullable + @Override + public SegmentChangeRequestLoad apply(DataSegment input) + { + return new SegmentChangeRequestLoad(input); + } + } + ); + + SettableFuture future = SettableFuture.create(); + future.set(SegmentChangeRequestsSnapshot.success(changes.getLastCounter(), Lists.newArrayList(segments))); + return future; + } + } else { + return changes.getRequestsSince(counter); + } } private String makeServedSegmentPath() { // server.getName() is already in the zk path - return makeServedSegmentPath(UUIDUtils.generateUuid( - server.getHost(), - server.getType(), - server.getTier(), - new DateTime().toString() - )); + return makeServedSegmentPath( + UUIDUtils.generateUuid( + server.getHost(), + server.getType().toString(), + server.getTier(), + new DateTime().toString() + ) + ); } private String makeServedSegmentPath(String zNode) diff --git a/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java similarity index 77% rename from server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java rename to server/src/main/java/io/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java index cdcc5bbf5766..6b2af0f6d82f 100644 --- a/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java @@ -22,19 +22,17 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; - +import com.google.inject.Inject; import io.druid.curator.announcement.Announcer; -import io.druid.java.util.common.lifecycle.LifecycleStart; -import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.java.util.common.logger.Logger; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; /** */ -public abstract class AbstractDataSegmentAnnouncer implements DataSegmentAnnouncer +public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnouncer { - private static final Logger log = new Logger(AbstractDataSegmentAnnouncer.class); + private static final Logger log = new Logger(CuratorDataSegmentServerAnnouncer.class); private final DruidServerMetadata server; private final ZkPathsConfig config; @@ -43,9 +41,10 @@ public abstract class AbstractDataSegmentAnnouncer implements DataSegmentAnnounc private final Object lock = new Object(); - private volatile boolean started = false; + private volatile boolean announced = false; - protected AbstractDataSegmentAnnouncer( + @Inject + public CuratorDataSegmentServerAnnouncer( DruidServerMetadata server, ZkPathsConfig config, Announcer announcer, @@ -58,11 +57,11 @@ protected AbstractDataSegmentAnnouncer( this.jsonMapper = jsonMapper; } - @LifecycleStart - public void start() + @Override + public void announce() { synchronized (lock) { - if (started) { + if (announced) { return; } @@ -75,22 +74,23 @@ public void start() throw Throwables.propagate(e); } - started = true; + announced = true; } } - @LifecycleStop - public void stop() + @Override + public void unannounce() { synchronized (lock) { - if (!started) { + if (!announced) { return; } - log.info("Stopping %s with config[%s]", getClass(), config); - announcer.unannounce(makeAnnouncementPath()); + final String path = makeAnnouncementPath(); + log.info("Unannouncing self[%s] at [%s]", server, path); + announcer.unannounce(path); - started = false; + announced = false; } } diff --git a/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java index ef6eafab6a7c..d6e8c4a27a8d 100644 --- a/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java @@ -32,9 +32,4 @@ public interface DataSegmentAnnouncer public void announceSegments(Iterable segments) throws IOException; public void unannounceSegments(Iterable segments) throws IOException; - - /** - * @return true if the segment was already announced, otherwise false - */ - public boolean isAnnounced(DataSegment segment); } diff --git a/server/src/main/java/io/druid/server/coordination/DataSegmentServerAnnouncer.java b/server/src/main/java/io/druid/server/coordination/DataSegmentServerAnnouncer.java new file mode 100644 index 000000000000..791c62c57a70 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/DataSegmentServerAnnouncer.java @@ -0,0 +1,28 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordination; + +/** + */ +public interface DataSegmentServerAnnouncer +{ + void announce(); + void unannounce(); +} diff --git a/server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java b/server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java index 9094d1009980..7c4ff251a9ba 100644 --- a/server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java +++ b/server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java @@ -30,7 +30,7 @@ public class DruidServerMetadata private final String host; private final long maxSize; private final String tier; - private final String type; + private final ServerType type; private final int priority; @JsonCreator @@ -38,7 +38,7 @@ public DruidServerMetadata( @JsonProperty("name") String name, @JsonProperty("host") String host, @JsonProperty("maxSize") long maxSize, - @JsonProperty("type") String type, + @JsonProperty("type") ServerType type, @JsonProperty("tier") String tier, @JsonProperty("priority") int priority ) @@ -76,7 +76,7 @@ public String getTier() } @JsonProperty - public String getType() + public ServerType getType() { return type; } @@ -87,9 +87,9 @@ public int getPriority() return priority; } - public boolean isAssignable() + public boolean segmentReplicatable() { - return getType().equalsIgnoreCase("historical") || getType().equalsIgnoreCase("bridge"); + return type.isSegmentReplicationTarget(); } @Override diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestHistory.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestHistory.java new file mode 100644 index 000000000000..e8389b93ece8 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestHistory.java @@ -0,0 +1,354 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.AbstractFuture; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.druid.common.utils.StringUtils; +import io.druid.java.util.common.IAE; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * This class keeps a bounded list of segment updates made on the server such as adding/dropping segments. + * + * Clients call addSegmentChangeRequest(DataSegmentChangeRequest) or addSegmentChangeRequests(List) + * to add segment updates. + * + * Clients call ListenableFuture getRequestsSince(final Counter counter) to get segment + * updates since given counter. + */ +public class SegmentChangeRequestHistory +{ + private static int MAX_SIZE = 1000; + + private final int maxSize; + + private final CircularBuffer changes; + + @VisibleForTesting + final LinkedHashMap waitingFutures; + + private final ExecutorService singleThreadedExecutor; + private final Runnable resolveWaitingFuturesRunnable; + + public SegmentChangeRequestHistory() + { + this(MAX_SIZE); + } + + public SegmentChangeRequestHistory(int maxSize) + { + this.maxSize = maxSize; + this.changes = new CircularBuffer(maxSize); + + this.waitingFutures = new LinkedHashMap<>(); + + this.resolveWaitingFuturesRunnable = new Runnable() + { + @Override + public void run() + { + resolveWaitingFutures(); + } + }; + + this.singleThreadedExecutor = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat( + "SegmentChangeRequestHistory" + ) + .build() + ); + } + + + + /** + * Add batch of segment changes update. + */ + public synchronized void addSegmentChangeRequests(List requests) + { + for (DataSegmentChangeRequest request : requests) { + changes.add(new Holder(request, getLastCounter().inc())); + } + + singleThreadedExecutor.execute(resolveWaitingFuturesRunnable); + } + + /** + * Add single segment change update. + */ + public synchronized void addSegmentChangeRequest(DataSegmentChangeRequest request) + { + addSegmentChangeRequests(ImmutableList.of(request)); + } + + /** + * Returns a Future that , on completion, returns list of segment updates and associated counter. + * If there are no update since given counter then Future completion waits till an updates is provided. + * + * If counter is older than max number of changes maintained then SegmentChangeRequestsSnapshot is returned + * with resetCounter set to True. + * + * If there were no updates to provide immediately then a future is created and returned to caller. This future + * is added to the "waitingFutures" list and all the futures in the list get resolved as soon as a segment + * update is provided. + */ + public synchronized ListenableFuture getRequestsSince(final Counter counter) + { + final CustomSettableFuture future = new CustomSettableFuture(waitingFutures); + + if (counter.counter < 0) { + future.setException(new IAE("counter[%s] must be >= 0", counter)); + return future; + } + + Counter lastCounter = getLastCounter(); + + if (counter.counter == lastCounter.counter) { + if (!counter.matches(lastCounter)) { + future.setException(new IAE("counter[%s] failed to match with [%s]", counter, lastCounter)); + } else { + synchronized (waitingFutures) { + waitingFutures.put(future, counter); + } + } + } else { + try { + future.set(getRequestsSinceWithoutWait(counter)); + } catch (Exception ex) { + future.setException(ex); + } + } + + return future; + } + + private synchronized SegmentChangeRequestsSnapshot getRequestsSinceWithoutWait(final Counter counter) + { + Counter lastCounter = getLastCounter(); + + if (counter.counter >= lastCounter.counter) { + throw new IAE("counter[%s] >= last counter[%s]", counter, lastCounter); + } else if (lastCounter.counter - counter.counter >= maxSize) { + // Note: counter reset is requested when client ask for "maxSize" number of changes even if all those changes + // are present in the history because one extra elements is needed to match the counter hash. + return SegmentChangeRequestsSnapshot.fail( + StringUtils.safeFormat( + "can't serve request, not enough history is kept. given counter [%s] and current last counter [%s]", + counter, + lastCounter + ) + ); + } else { + int changeStartIndex = (int) (counter.counter + changes.size() - lastCounter.counter); + + Counter counterToMatch = counter.counter == 0 ? Counter.ZERO : changes.get(changeStartIndex - 1).counter; + if (!counterToMatch.matches(counter)) { + throw new IAE("counter[%s] failed to match with [%s]", counter, counterToMatch); + } + + List result = new ArrayList<>(); + for (int i = changeStartIndex; i < changes.size(); i++) { + result.add(changes.get(i).changeRequest); + } + + return SegmentChangeRequestsSnapshot.success(changes.get(changes.size() - 1).counter, result); + } + } + + private void resolveWaitingFutures() + { + final LinkedHashMap waitingFuturesCopy = new LinkedHashMap<>(); + synchronized (waitingFutures) { + waitingFuturesCopy.putAll(waitingFutures); + waitingFutures.clear(); + } + + for (Map.Entry e : waitingFuturesCopy.entrySet()) { + try { + e.getKey().set(getRequestsSinceWithoutWait(e.getValue())); + } catch (Exception ex) { + e.getKey().setException(ex); + } + } + } + + public synchronized Counter getLastCounter() + { + if (changes.size() > 0) { + return changes.get(changes.size() - 1).counter; + } else { + return Counter.ZERO; + } + } + + private static class Holder + { + private final DataSegmentChangeRequest changeRequest; + private final Counter counter; + + public Holder(DataSegmentChangeRequest changeRequest, Counter counter) + { + this.changeRequest = changeRequest; + this.counter = counter; + } + } + + public static class Counter + { + public static final Counter ZERO = new Counter(0); + + private final long counter; + private final long hash; + + public Counter(long counter) + { + this(counter, System.currentTimeMillis()); + } + + @JsonCreator + public Counter( + @JsonProperty("counter") long counter, + @JsonProperty("hash") long hash + ) + { + this.counter = counter; + this.hash = hash; + } + + @JsonProperty + public long getCounter() + { + return counter; + } + + @JsonProperty + public long getHash() + { + return hash; + } + + public Counter inc() + { + return new Counter(counter + 1); + } + + public boolean matches(Counter other) + { + return this.counter == other.counter && this.hash == other.hash; + } + + @Override + public String toString() + { + return "Counter{" + + "counter=" + counter + + ", hash=" + hash + + '}'; + } + } + + // Future with cancel() implementation to remove it from "waitingFutures" list + private static class CustomSettableFuture extends AbstractFuture + { + private final LinkedHashMap waitingFutures; + + private CustomSettableFuture(LinkedHashMap waitingFutures) + { + this.waitingFutures = waitingFutures; + } + + @Override + public boolean set(SegmentChangeRequestsSnapshot value) + { + return super.set(value); + } + + @Override + public boolean setException(Throwable throwable) + { + return super.setException(throwable); + } + + @Override + public boolean cancel(boolean interruptIfRunning) + { + synchronized (waitingFutures) { + waitingFutures.remove(this); + } + return true; + } + } + + static class CircularBuffer + { + private final E[] buffer; + + private int start = 0; + private int size = 0; + + CircularBuffer(int capacity) + { + buffer = (E[]) new Object[capacity]; + } + + void add(E item) + { + buffer[start++] = item; + + if (start >= buffer.length) { + start = 0; + } + + if (size < buffer.length) { + size++; + } + } + + E get(int index) + { + Preconditions.checkArgument(index >= 0 && index < size, "invalid index"); + + int bufferIndex = (start-size+index) % buffer.length; + if (bufferIndex < 0) { + bufferIndex += buffer.length; + } + return buffer[bufferIndex]; + } + + int size() + { + return size; + } + } +} diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestsSnapshot.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestsSnapshot.java new file mode 100644 index 000000000000..893e04e474ce --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestsSnapshot.java @@ -0,0 +1,108 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.List; + +/** + * Return type of SegmentChangeRequestHistory.getRequestsSince(counter). + */ +public class SegmentChangeRequestsSnapshot +{ + //if true, that means caller should reset the counter and request again. + private final boolean resetCounter; + + //cause for reset if resetCounter is true + private final String resetCause; + + //segments requests delta since counter, if resetCounter if false + private final SegmentChangeRequestHistory.Counter counter; + private final List requests; + + @JsonCreator + public SegmentChangeRequestsSnapshot( + @JsonProperty("resetCounter") boolean resetCounter, + @JsonProperty("resetCause") String resetCause, + @JsonProperty("counter") SegmentChangeRequestHistory.Counter counter, + @JsonProperty("requests") List requests + ) + { + this.resetCounter = resetCounter; + this.resetCause = resetCause; + + if (resetCounter) { + Preconditions.checkNotNull(resetCause, "NULL resetCause when resetCounter is true."); + } + + + this.counter = counter; + this.requests = requests; + } + + public static SegmentChangeRequestsSnapshot success(SegmentChangeRequestHistory.Counter counter, + List requests) + { + return new SegmentChangeRequestsSnapshot(false, null, counter, requests); + } + + public static SegmentChangeRequestsSnapshot fail(String resetCause) + { + return new SegmentChangeRequestsSnapshot(true, resetCause, null, null); + } + + @JsonProperty + public boolean isResetCounter() + { + return resetCounter; + } + + @JsonProperty + public String getResetCause() + { + return resetCause; + } + + @JsonProperty + public SegmentChangeRequestHistory.Counter getCounter() + { + return counter; + } + + @JsonProperty + public List getRequests() + { + return requests; + } + + @Override + public String toString() + { + return "SegmentChangeRequestsSnapshot{" + + "resetCounter=" + resetCounter + + ", resetCause='" + resetCause + '\'' + + ", counter=" + counter + + ", requests=" + requests + + '}'; + } +} diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 65d94aab0d16..9b906325cbad 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -22,14 +22,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Iterables; -import com.google.common.collect.Ordering; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.CachingQueryRunner; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; -import io.druid.collections.CountingMap; import io.druid.guice.annotations.BackgroundCaching; import io.druid.guice.annotations.Processing; import io.druid.guice.annotations.Smile; @@ -55,10 +53,7 @@ import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; import io.druid.segment.ReferenceCountingSegment; -import io.druid.segment.Segment; -import io.druid.segment.loading.SegmentLoader; -import io.druid.segment.loading.SegmentLoadingException; -import io.druid.timeline.DataSegment; +import io.druid.server.SegmentManager; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -66,10 +61,8 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; +import java.util.Collections; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; @@ -78,32 +71,27 @@ public class ServerManager implements QuerySegmentWalker { private static final EmittingLogger log = new EmittingLogger(ServerManager.class); - private final Object lock = new Object(); - private final SegmentLoader segmentLoader; private final QueryRunnerFactoryConglomerate conglomerate; private final ServiceEmitter emitter; private final ExecutorService exec; private final ExecutorService cachingExec; - private final Map> dataSources; - private final CountingMap dataSourceSizes = new CountingMap(); - private final CountingMap dataSourceCounts = new CountingMap(); private final Cache cache; private final ObjectMapper objectMapper; private final CacheConfig cacheConfig; + private final SegmentManager segmentManager; @Inject public ServerManager( - SegmentLoader segmentLoader, QueryRunnerFactoryConglomerate conglomerate, ServiceEmitter emitter, @Processing ExecutorService exec, @BackgroundCaching ExecutorService cachingExec, @Smile ObjectMapper objectMapper, Cache cache, - CacheConfig cacheConfig + CacheConfig cacheConfig, + SegmentManager segmentManager ) { - this.segmentLoader = segmentLoader; this.conglomerate = conglomerate; this.emitter = emitter; @@ -112,137 +100,8 @@ public ServerManager( this.cache = cache; this.objectMapper = objectMapper; - this.dataSources = new HashMap<>(); this.cacheConfig = cacheConfig; - } - - public Map getDataSourceSizes() - { - synchronized (dataSourceSizes) { - return dataSourceSizes.snapshot(); - } - } - - public Map getDataSourceCounts() - { - synchronized (dataSourceCounts) { - return dataSourceCounts.snapshot(); - } - } - - public boolean isSegmentCached(final DataSegment segment) throws SegmentLoadingException - { - return segmentLoader.isSegmentLoaded(segment); - } - - /** - * Load a single segment. - * - * @param segment segment to load - * - * @return true if the segment was newly loaded, false if it was already loaded - * - * @throws SegmentLoadingException if the segment cannot be loaded - */ - public boolean loadSegment(final DataSegment segment) throws SegmentLoadingException - { - final Segment adapter; - try { - adapter = segmentLoader.getSegment(segment); - } - catch (SegmentLoadingException e) { - try { - segmentLoader.cleanup(segment); - } - catch (SegmentLoadingException e1) { - // ignore - } - throw e; - } - - if (adapter == null) { - throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); - } - - synchronized (lock) { - String dataSource = segment.getDataSource(); - VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); - - if (loadedIntervals == null) { - loadedIntervals = new VersionedIntervalTimeline<>(Ordering.natural()); - dataSources.put(dataSource, loadedIntervals); - } - - PartitionHolder entry = loadedIntervals.findEntry( - segment.getInterval(), - segment.getVersion() - ); - if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { - log.warn("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); - return false; - } - - loadedIntervals.add( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter)) - ); - synchronized (dataSourceSizes) { - dataSourceSizes.add(dataSource, segment.getSize()); - } - synchronized (dataSourceCounts) { - dataSourceCounts.add(dataSource, 1L); - } - return true; - } - } - - public void dropSegment(final DataSegment segment) throws SegmentLoadingException - { - String dataSource = segment.getDataSource(); - synchronized (lock) { - VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); - - if (loadedIntervals == null) { - log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSource); - return; - } - - PartitionChunk removed = loadedIntervals.remove( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().createChunk((ReferenceCountingSegment) null) - ); - ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); - - if (oldQueryable != null) { - synchronized (dataSourceSizes) { - dataSourceSizes.add(dataSource, -segment.getSize()); - } - synchronized (dataSourceCounts) { - dataSourceCounts.add(dataSource, -1L); - } - - try { - log.info("Attempting to close segment %s", segment.getIdentifier()); - oldQueryable.close(); - } - catch (IOException e) { - log.makeAlert(e, "Exception closing segment") - .addData("dataSource", dataSource) - .addData("segmentId", segment.getIdentifier()) - .emit(); - } - } else { - log.info( - "Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.", - dataSource, - segment.getInterval(), - segment.getVersion() - ); - } - } - segmentLoader.cleanup(segment); + this.segmentManager = segmentManager; } @Override @@ -262,7 +121,9 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable timeline = dataSources.get(dataSourceName); + final VersionedIntervalTimeline timeline = segmentManager.getTimeline( + dataSourceName + ); if (timeline == null) { return new NoopQueryRunner(); @@ -351,7 +212,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable timeline = dataSources.get( + final VersionedIntervalTimeline timeline = segmentManager.getTimeline( dataSourceName ); @@ -381,11 +242,11 @@ public Iterable> apply(SegmentDescriptor input) final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); if (chunk == null) { - return Arrays.>asList(new ReportTimelineMissingSegmentQueryRunner(input)); + return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner(input)); } final ReferenceCountingSegment adapter = chunk.getObject(); - return Arrays.asList( + return Collections.singletonList( buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator) ); } diff --git a/server/src/main/java/io/druid/server/coordination/ServerType.java b/server/src/main/java/io/druid/server/coordination/ServerType.java new file mode 100644 index 000000000000..f0acf83edd6e --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/ServerType.java @@ -0,0 +1,91 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +/** + * This enum represents types of druid services that hold segments. + *

+ * These types are externally visible (e.g., from the output of /druid/coordinator/v1/servers). + *

+ * For backwards compatibility, when presenting these types externally, the toString() representation + * of the enum should be used. + *

+ * The toString() method converts the enum name() to lowercase and replaces underscores with hyphens, + * which is the format expected for the server type string prior to the patch that introduced ServerType: + * https://github.com/druid-io/druid/pull/4148 + */ +public enum ServerType +{ + HISTORICAL, + BRIDGE, + INDEXER_EXECUTOR { + @Override + public boolean isSegmentReplicationTarget() + { + return false; + } + }, + + REALTIME { + @Override + public boolean isSegmentReplicationTarget() + { + return false; + } + }; + + /** + * Indicates this type of node is able to be a target of segment replication. + * + * @return true if it is available for replication + * + * @see io.druid.server.coordinator.rules.LoadRule + */ + public boolean isSegmentReplicationTarget() + { + return true; + } + + /** + * Indicates this type of node is able to be a target of segment broadcast. + * + * @return true if it is available for broadcast. + */ + public boolean isSegmentBroadcastTarget() + { + return true; + } + + @JsonCreator + public static ServerType fromString(String type) + { + return ServerType.valueOf(type.toUpperCase().replace("-", "_")); + } + + @Override + @JsonValue + public String toString() + { + return name().toLowerCase().replace("_", "-"); + } +} diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index 0fd4c3f60ac2..6286e516b611 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -34,6 +34,7 @@ import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoadingException; +import io.druid.server.SegmentManager; import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; @@ -72,7 +73,8 @@ public class ZkCoordinator implements DataSegmentChangeHandler private final DruidServerMetadata me; private final CuratorFramework curator; private final DataSegmentAnnouncer announcer; - private final ServerManager serverManager; + private final DataSegmentServerAnnouncer serverAnnouncer; + private final SegmentManager segmentManager; private final ScheduledExecutorService exec; private final ConcurrentSkipListSet segmentsToDelete; @@ -87,8 +89,9 @@ public ZkCoordinator( ZkPathsConfig zkPaths, DruidServerMetadata me, DataSegmentAnnouncer announcer, + DataSegmentServerAnnouncer serverAnnouncer, CuratorFramework curator, - ServerManager serverManager, + SegmentManager segmentManager, ScheduledExecutorFactory factory ) { @@ -98,7 +101,8 @@ public ZkCoordinator( this.me = me; this.curator = curator; this.announcer = announcer; - this.serverManager = serverManager; + this.serverAnnouncer = serverAnnouncer; + this.segmentManager = segmentManager; this.exec = factory.create(1, "ZkCoordinator-Exec--%d"); this.segmentsToDelete = new ConcurrentSkipListSet<>(); @@ -132,6 +136,7 @@ public void start() throws IOException curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient()); loadLocalCache(); + serverAnnouncer.announce(); loadQueueCache.getListenable().addListener( new PathChildrenCacheListener() @@ -226,6 +231,7 @@ public void stop() try { loadQueueCache.close(); + serverAnnouncer.unannounce(); } catch (Exception e) { throw Throwables.propagate(e); @@ -252,12 +258,17 @@ public void loadLocalCache() List cachedSegments = Lists.newArrayList(); File[] segmentsToLoad = baseDir.listFiles(); + int ignored = 0; for (int i = 0; i < segmentsToLoad.length; i++) { File file = segmentsToLoad[i]; log.info("Loading segment cache file [%d/%d][%s].", i, segmentsToLoad.length, file); try { - DataSegment segment = jsonMapper.readValue(file, DataSegment.class); - if (serverManager.isSegmentCached(segment)) { + final DataSegment segment = jsonMapper.readValue(file, DataSegment.class); + + if (!segment.getIdentifier().equals(file.getName())) { + log.warn("Ignoring cache file[%s] for segment[%s].", file.getPath(), segment.getIdentifier()); + ignored++; + } else if (segmentManager.isSegmentCached(segment)) { cachedSegments.add(segment); } else { log.warn("Unable to find cache file for %s. Deleting lookup entry", segment.getIdentifier()); @@ -275,6 +286,12 @@ public void loadLocalCache() } } + if (ignored > 0) { + log.makeAlert("Ignored misnamed segment cache files on startup.") + .addData("numIgnored", ignored) + .emit(); + } + addSegments( cachedSegments, new DataSegmentChangeCallback() @@ -294,7 +311,7 @@ public DataSegmentChangeHandler getDataSegmentChangeHandler() } /** - * Load a single segment. If the segment is loaded succesfully, this function simply returns. Otherwise it will + * Load a single segment. If the segment is loaded successfully, this function simply returns. Otherwise it will * throw a SegmentLoadingException * * @throws SegmentLoadingException @@ -303,7 +320,7 @@ private void loadSegment(DataSegment segment, DataSegmentChangeCallback callback { final boolean loaded; try { - loaded = serverManager.loadSegment(segment); + loaded = segmentManager.loadSegment(segment); } catch (Exception e) { removeSegment(segment, callback); @@ -349,13 +366,11 @@ each time when addSegment() is called, it has to wait for the lock in order to m } } loadSegment(segment, callback); - if (!announcer.isAnnounced(segment)) { - try { - announcer.announceSegment(segment); - } - catch (IOException e) { - throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier()); - } + try { + announcer.announceSegment(segment); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Failed to announce segment[%s]", segment.getIdentifier()); } } catch (SegmentLoadingException e) { @@ -397,14 +412,12 @@ public void run() segment.getIdentifier() ); loadSegment(segment, callback); - if (!announcer.isAnnounced(segment)) { - try { - backgroundSegmentAnnouncer.announceSegment(segment); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new SegmentLoadingException(e, "Loading Interrupted"); - } + try { + backgroundSegmentAnnouncer.announceSegment(segment); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new SegmentLoadingException(e, "Loading Interrupted"); } } catch (SegmentLoadingException e) { @@ -465,7 +478,7 @@ public void run() try { synchronized (lock) { if (segmentsToDelete.remove(segment)) { - serverManager.dropSegment(segment); + segmentManager.dropSegment(segment); File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { @@ -595,7 +608,7 @@ public void finishAnnouncing() throws SegmentLoadingException throw new SegmentLoadingException(e, "Failed to announce segments[%s]", queue); } - // get any exception that may have been thrown in background annoucing + // get any exception that may have been thrown in background announcing try { // check in case intervalMillis is <= 0 if (startedAnnouncing != null) { diff --git a/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java index 5ff435299315..4589b141fc2e 100644 --- a/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java @@ -20,64 +20,98 @@ package io.druid.server.coordinator; import com.google.common.collect.Maps; -import io.druid.collections.CountingMap; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; +import java.util.Collections; import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; +import java.util.Set; +import java.util.function.ObjLongConsumer; /** */ public class CoordinatorStats { - private final Map> perTierStats; - private final CountingMap globalStats; + private final Map> perTierStats; + private final Object2LongOpenHashMap globalStats; public CoordinatorStats() { perTierStats = Maps.newHashMap(); - globalStats = new CountingMap(); + globalStats = new Object2LongOpenHashMap<>(); } - public Map> getPerTierStats() + public boolean hasPerTierStats() { - return perTierStats; + return !perTierStats.isEmpty(); } - public CountingMap getGlobalStats() + public Set getTiers(final String statName) { - return globalStats; - } - - public void addToTieredStat(String statName, String tier, long value) - { - CountingMap theStat = perTierStats.get(statName); + final Object2LongOpenHashMap theStat = perTierStats.get(statName); if (theStat == null) { - theStat = new CountingMap(); - perTierStats.put(statName, theStat); + return Collections.emptySet(); } - theStat.add(tier, value); + return Collections.unmodifiableSet(theStat.keySet()); } - public void addToGlobalStat(String statName, long value) + /** + * + * @param statName the name of the statistics + * @param tier the tier + * @return the value for the statistics {@code statName} under {@code tier} tier + * @throws NullPointerException if {@code statName} is not found + */ + public long getTieredStat(final String statName, final String tier) { - globalStats.add(statName, value); + return perTierStats.get(statName).getLong(tier); } - public CoordinatorStats accumulate(CoordinatorStats stats) + public void forEachTieredStat(final String statName, final ObjLongConsumer consumer) { - for (Map.Entry> entry : stats.perTierStats.entrySet()) { - CountingMap theStat = perTierStats.get(entry.getKey()); - if (theStat == null) { - theStat = new CountingMap(); - perTierStats.put(entry.getKey(), theStat); - } - for (Map.Entry tiers : entry.getValue().entrySet()) { - theStat.add(tiers.getKey(), tiers.getValue().get()); + final Object2LongOpenHashMap theStat = perTierStats.get(statName); + if (theStat != null) { + for (final Object2LongMap.Entry entry : theStat.object2LongEntrySet()) { + consumer.accept(entry.getKey(), entry.getLongValue()); } } - for (Map.Entry entry : stats.globalStats.entrySet()) { - globalStats.add(entry.getKey(), entry.getValue().get()); + } + + public long getGlobalStat(final String statName) + { + return globalStats.getLong(statName); + } + + public void addToTieredStat(final String statName, final String tier, final long value) + { + perTierStats.computeIfAbsent(statName, ignored -> new Object2LongOpenHashMap<>()) + .addTo(tier, value); + } + + public void addToGlobalStat(final String statName, final long value) + { + globalStats.addTo(statName, value); + } + + public CoordinatorStats accumulate(final CoordinatorStats stats) + { + stats.perTierStats.forEach( + (final String statName, final Object2LongOpenHashMap urStat) -> { + + final Object2LongOpenHashMap myStat = perTierStats.computeIfAbsent( + statName, ignored -> new Object2LongOpenHashMap<>() + ); + + for (final Object2LongMap.Entry entry : urStat.object2LongEntrySet()) { + myStat.addTo(entry.getKey(), entry.getLongValue()); + } + } + ); + + for (final Object2LongMap.Entry entry : stats.globalStats.object2LongEntrySet()) { + globalStats.addTo(entry.getKey(), entry.getLongValue()); } + return this; } } diff --git a/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java index 014bc231fb78..e0e4b2d45b52 100644 --- a/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java +++ b/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.ThreadLocalRandom; public class CostBalancerStrategy implements BalancerStrategy { @@ -211,6 +212,7 @@ static double computeJointSegmentsCost(final DataSegment segment, final Iterable } + @Override public BalancerSegmentHolder pickSegmentToMove(final List serverHolders) { ReservoirSegmentSampler sampler = new ReservoirSegmentSampler(); @@ -362,7 +364,7 @@ public Pair call() throws Exception } // Randomly choose a server from the best servers - bestServer = bestServers.get((int) Math.random() * bestServers.size()); + bestServer = bestServers.get(ThreadLocalRandom.current().nextInt(bestServers.size())); } catch (Exception e) { log.makeAlert(e, "Cost Balancer Multithread strategy wasn't able to complete cost computation.").emit(); diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCluster.java b/server/src/main/java/io/druid/server/coordinator/DruidCluster.java index 75ec66e5f392..20561bee8550 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCluster.java @@ -19,15 +19,18 @@ package io.druid.server.coordinator; -import com.google.common.collect.Maps; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Ordering; import io.druid.client.ImmutableDruidServer; +import io.druid.java.util.common.IAE; +import javax.annotation.Nullable; import java.util.Collection; -import java.util.List; +import java.util.HashMap; +import java.util.HashSet; import java.util.Map; -import java.util.stream.Collectors; +import java.util.Set; /** * Contains a representation of the current state of the cluster by tier. @@ -35,67 +38,109 @@ */ public class DruidCluster { - private final Map> cluster; + private final Set realtimes; + private final Map> historicals; public DruidCluster() { - this.cluster = Maps.newHashMap(); + this.realtimes = new HashSet<>(); + this.historicals = new HashMap<>(); } - public DruidCluster(Map> cluster) + @VisibleForTesting + public DruidCluster( + @Nullable Set realtimes, + Map> historicals + ) { - this.cluster = cluster; + this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes); + this.historicals = historicals; } public void add(ServerHolder serverHolder) { - ImmutableDruidServer server = serverHolder.getServer(); - MinMaxPriorityQueue tierServers = cluster.get(server.getTier()); - if (tierServers == null) { - tierServers = MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create(); - cluster.put(server.getTier(), tierServers); + switch (serverHolder.getServer().getType()) { + case HISTORICAL: + addHistorical(serverHolder); + break; + case REALTIME: + addRealtime(serverHolder); + break; + case BRIDGE: + addHistorical(serverHolder); + break; + case INDEXER_EXECUTOR: + throw new IAE("unsupported server type[%s]", serverHolder.getServer().getType()); + default: + throw new IAE("unknown server type[%s]", serverHolder.getServer().getType()); } + } + + private void addRealtime(ServerHolder serverHolder) + { + realtimes.add(serverHolder); + } + + private void addHistorical(ServerHolder serverHolder) + { + final ImmutableDruidServer server = serverHolder.getServer(); + final MinMaxPriorityQueue tierServers = historicals.computeIfAbsent( + server.getTier(), + k -> MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create() + ); tierServers.add(serverHolder); } - public Map> getCluster() + public Set getRealtimes() + { + return realtimes; + } + + public Map> getHistoricals() { - return cluster; + return historicals; } public Iterable getTierNames() { - return cluster.keySet(); + return historicals.keySet(); } - public MinMaxPriorityQueue getServersByTier(String tier) + public MinMaxPriorityQueue getHistoricalsByTier(String tier) { - return cluster.get(tier); + return historicals.get(tier); } - public List getAllServers() + public Collection getAllServers() { - return cluster.values().stream().flatMap(Collection::stream).collect(Collectors.toList()); + return historicals.values().stream() + .flatMap(Collection::stream) + .collect(() -> realtimes, Set::add, Set::addAll); } - public Iterable> getSortedServersByTier() + public Iterable> getSortedHistoricalsByTier() { - return cluster.values(); + return historicals.values(); } public boolean isEmpty() { - return cluster.isEmpty(); + return historicals.isEmpty() && realtimes.isEmpty(); } - public boolean hasTier(String tier) + public boolean hasHistoricals() { - MinMaxPriorityQueue servers = cluster.get(tier); - return (servers == null) || servers.isEmpty(); + return !historicals.isEmpty(); + } + + public boolean hasRealtimes() + { + return !realtimes.isEmpty(); } - public MinMaxPriorityQueue get(String tier) + public boolean hasTier(String tier) { - return cluster.get(tier); + MinMaxPriorityQueue servers = historicals.get(tier); + return (servers == null) || servers.isEmpty(); } } diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index b36f0061049c..4ca6333b3d38 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -39,7 +39,6 @@ import io.druid.client.ImmutableDruidServer; import io.druid.client.ServerInventoryView; import io.druid.client.indexing.IndexingServiceClient; -import io.druid.collections.CountingMap; import io.druid.common.config.JacksonConfigManager; import io.druid.concurrent.Execs; import io.druid.curator.discovery.ServiceAnnouncer; @@ -70,6 +69,8 @@ import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; import org.apache.curator.framework.recipes.leader.LeaderLatchListener; @@ -117,7 +118,7 @@ public Interval apply(DataSegment segment) private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; private final MetadataSegmentManager metadataSegmentManager; - private final ServerInventoryView serverInventoryView; + private final ServerInventoryView serverInventoryView; private final MetadataRuleManager metadataRuleManager; private final CuratorFramework curator; private final ServiceEmitter emitter; @@ -230,65 +231,67 @@ public Map getLoadManagementPeons() return loadManagementPeons; } - public Map> getReplicationStatus() + public Map> getReplicationStatus() { - final Map> retVal = Maps.newHashMap(); + final Map> retVal = Maps.newHashMap(); if (segmentReplicantLookup == null) { return retVal; } final DateTime now = new DateTime(); - for (DataSegment segment : getAvailableDataSegments()) { - List rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource()); - for (Rule rule : rules) { - if (rule instanceof LoadRule && rule.appliesTo(segment, now)) { - for (Map.Entry entry : ((LoadRule) rule).getTieredReplicants().entrySet()) { - CountingMap dataSourceMap = retVal.get(entry.getKey()); - if (dataSourceMap == null) { - dataSourceMap = new CountingMap<>(); - retVal.put(entry.getKey(), dataSourceMap); - } - int diff = Math.max( - entry.getValue() - segmentReplicantLookup.getTotalReplicants(segment.getIdentifier(), entry.getKey()), - 0 - ); - dataSourceMap.add(segment.getDataSource(), diff); - } - break; + for (final DataSegment segment : getAvailableDataSegments()) { + final List rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource()); + + for (final Rule rule : rules) { + if (!(rule instanceof LoadRule && rule.appliesTo(segment, now))) { + continue; } + + ((LoadRule) rule) + .getTieredReplicants() + .forEach( + (final String tier, final Integer ruleReplicants) -> { + int currentReplicants = segmentReplicantLookup.getTotalReplicants(segment.getIdentifier(), tier); + retVal + .computeIfAbsent(tier, ignored -> new Object2LongOpenHashMap<>()) + .addTo(segment.getDataSource(), Math.max(ruleReplicants - currentReplicants, 0)); + } + ); } } return retVal; } - public CountingMap getSegmentAvailability() + + public Object2LongMap getSegmentAvailability() { - final CountingMap retVal = new CountingMap<>(); + final Object2LongOpenHashMap retVal = new Object2LongOpenHashMap<>(); if (segmentReplicantLookup == null) { return retVal; } for (DataSegment segment : getAvailableDataSegments()) { - int available = (segmentReplicantLookup.getTotalReplicants(segment.getIdentifier()) == 0) ? 0 : 1; - retVal.add(segment.getDataSource(), 1 - available); + if (segmentReplicantLookup.getTotalReplicants(segment.getIdentifier()) == 0) { + retVal.addTo(segment.getDataSource(), 1); + } else { + retVal.addTo(segment.getDataSource(), 0); + } } return retVal; } - CountingMap getLoadPendingDatasources() + boolean hasLoadPending(final String dataSource) { - final CountingMap retVal = new CountingMap<>(); - for (LoadQueuePeon peon : loadManagementPeons.values()) { - for (DataSegment segment : peon.getSegmentsToLoad()) { - retVal.add(segment.getDataSource(), 1); - } - } - return retVal; + return loadManagementPeons + .values() + .stream() + .flatMap((final LoadQueuePeon peon) -> peon.getSegmentsToLoad().stream()) + .anyMatch((final DataSegment segment) -> segment.getDataSource().equals(dataSource)); } public Map getLoadStatus() @@ -362,9 +365,9 @@ public String getCurrentLeader() } public void moveSegment( - ImmutableDruidServer fromServer, - ImmutableDruidServer toServer, - String segmentName, + final ImmutableDruidServer fromServer, + final ImmutableDruidServer toServer, + final String segmentName, final LoadPeonCallback callback ) { @@ -405,10 +408,6 @@ public void moveSegment( toServer.getName() ), segmentName ); - final String toServedSegPath = ZKPaths.makePath( - ZKPaths.makePath(serverInventoryView.getInventoryManagerConfig().getInventoryPath(), toServer.getName()), - segmentName - ); loadPeon.loadSegment( segment, @@ -418,7 +417,7 @@ public void moveSegment( public void execute() { try { - if (curator.checkExists().forPath(toServedSegPath) != null && + if (serverInventoryView.isSegmentLoadedByServer(toServer.getName(), segment) && curator.checkExists().forPath(toLoadQueueSegPath) == null && !dropPeon.getSegmentsToDrop().contains(segment)) { dropPeon.dropSegment(segment, callback); @@ -755,7 +754,7 @@ public boolean apply( DruidServer input ) { - return input.isAssignable(); + return input.segmentReplicatable(); } } ).transform( diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index 3019284634c0..12dbf358d251 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.metamx.emitter.EmittingLogger; - import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.ScheduledExecutors; import io.druid.server.coordination.DataSegmentChangeRequest; @@ -38,8 +37,8 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.data.Stat; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.Callable; @@ -155,7 +154,7 @@ public void loadSegment( log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getIdentifier()); queuedSize.addAndGet(segment.getSize()); - segmentsToLoad.put(segment, new SegmentHolder(segment, LOAD, Arrays.asList(callback))); + segmentsToLoad.put(segment, new SegmentHolder(segment, LOAD, Collections.singletonList(callback))); } public void dropSegment( @@ -184,7 +183,7 @@ public void dropSegment( } log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier()); - segmentsToDrop.put(segment, new SegmentHolder(segment, DROP, Arrays.asList(callback))); + segmentsToDrop.put(segment, new SegmentHolder(segment, DROP, Collections.singletonList(callback))); } private void processSegmentChangeRequest() { @@ -243,6 +242,9 @@ public void process(WatchedEvent watchedEvent) throws Exception switch (watchedEvent.getType()) { case NodeDeleted: entryRemoved(watchedEvent.getPath()); + break; + default: + // do nothing } } } diff --git a/server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java b/server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java index 64b76b8fd070..785fce77feeb 100644 --- a/server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java +++ b/server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java @@ -38,7 +38,7 @@ public static SegmentReplicantLookup make(DruidCluster cluster) final Table segmentsInCluster = HashBasedTable.create(); final Table loadingSegments = HashBasedTable.create(); - for (MinMaxPriorityQueue serversByType : cluster.getSortedServersByTier()) { + for (MinMaxPriorityQueue serversByType : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serversByType) { ImmutableDruidServer server = serverHolder.getServer(); diff --git a/server/src/main/java/io/druid/server/coordinator/ServerHolder.java b/server/src/main/java/io/druid/server/coordinator/ServerHolder.java index 7f2f13693744..12b14fc3cba0 100644 --- a/server/src/main/java/io/druid/server/coordinator/ServerHolder.java +++ b/server/src/main/java/io/druid/server/coordinator/ServerHolder.java @@ -23,6 +23,8 @@ import io.druid.java.util.common.logger.Logger; import io.druid.timeline.DataSegment; +import java.util.Objects; + /** */ public class ServerHolder implements Comparable @@ -122,21 +124,20 @@ public boolean equals(Object o) ServerHolder that = (ServerHolder) o; - if (peon != null ? !peon.equals(that.peon) : that.peon != null) { + if (!this.server.getHost().equals(that.server.getHost())) { return false; } - if (server != null ? !server.equals(that.server) : that.server != null) { + + if (!this.server.getTier().equals(that.getServer().getTier())) { return false; } - return true; + return this.server.getType().equals(that.getServer().getType()); } @Override public int hashCode() { - int result = server != null ? server.hashCode() : 0; - result = 31 * result + (peon != null ? peon.hashCode() : 0); - return result; + return Objects.hash(server.getHost(), server.getTier(), server.getType()); } } diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index 67e750dd927f..d503c1d6a939 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -88,7 +88,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final int maxSegmentsToMove = params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(); for (Map.Entry> entry : - params.getDruidCluster().getCluster().entrySet()) { + params.getDruidCluster().getHistoricals().entrySet()) { String tier = entry.getKey(); if (currentlyMovingSegments.get(tier) == null) { diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index 68a83c3a9193..2819a51ec4cd 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -55,7 +55,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) DruidCluster cluster = params.getDruidCluster(); Map> timelines = Maps.newHashMap(); - for (MinMaxPriorityQueue serverHolders : cluster.getSortedServersByTier()) { + for (MinMaxPriorityQueue serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { ImmutableDruidServer server = serverHolder.getServer(); diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java index ee4fcc3dd799..89b4deb3175e 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java @@ -64,7 +64,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // This is done to prevent a race condition in which the coordinator would drop all segments if it started running // cleanup before it finished polling the metadata storage for available segments for the first time. if (!availableSegments.isEmpty()) { - for (MinMaxPriorityQueue serverHolders : cluster.getSortedServersByTier()) { + for (MinMaxPriorityQueue serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { ImmutableDruidServer server = serverHolder.getServer(); diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java index 55ab2f0a84d7..4ab3870c0f1e 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -19,13 +19,11 @@ package io.druid.server.coordinator.helper; -import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.DruidDataSource; import io.druid.client.ImmutableDruidServer; -import io.druid.collections.CountingMap; import io.druid.java.util.common.logger.Logger; import io.druid.query.DruidMetrics; import io.druid.server.coordinator.CoordinatorStats; @@ -35,10 +33,11 @@ import io.druid.server.coordinator.LoadQueuePeon; import io.druid.server.coordinator.ServerHolder; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.objects.Object2LongMap; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** */ @@ -47,29 +46,38 @@ public class DruidCoordinatorLogger implements DruidCoordinatorHelper private static final Logger log = new Logger(DruidCoordinatorLogger.class); private final DruidCoordinator coordinator; - public DruidCoordinatorLogger(DruidCoordinator coordinator) { + public DruidCoordinatorLogger(DruidCoordinator coordinator) + { this.coordinator = coordinator; } - private void emitTieredStats( + private void emitTieredStat( final ServiceEmitter emitter, final String metricName, - final Map statMap + final String tier, + final double value ) { - if (statMap != null) { - for (Map.Entry entry : statMap.entrySet()) { - String tier = entry.getKey(); - Number value = entry.getValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.TIER, tier) - .build( - metricName, value.doubleValue() - ) - ); - } - } + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.TIER, tier) + .build(metricName, value) + ); + } + + private void emitTieredStats( + final ServiceEmitter emitter, + final String metricName, + final CoordinatorStats stats, + final String statName + ) + { + stats.forEachTieredStat( + statName, + (final String tier, final long count) -> { + emitTieredStat(emitter, metricName, tier, count); + } + ); } @Override @@ -79,122 +87,98 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) CoordinatorStats stats = params.getCoordinatorStats(); ServiceEmitter emitter = params.getEmitter(); - Map assigned = stats.getPerTierStats().get("assignedCount"); - if (assigned != null) { - for (Map.Entry entry : assigned.entrySet()) { - log.info( - "[%s] : Assigned %s segments among %,d servers", - entry.getKey(), entry.getValue().get(), cluster.get(entry.getKey()).size() - ); - } - } + stats.forEachTieredStat( + "assignedCount", + (final String tier, final long count) -> { + log.info( + "[%s] : Assigned %s segments among %,d servers", + tier, count, cluster.getHistoricalsByTier(tier).size() + ); - emitTieredStats( - emitter, "segment/assigned/count", - assigned + emitTieredStat(emitter, "segment/assigned/count", tier, count); + } ); - Map dropped = stats.getPerTierStats().get("droppedCount"); - if (dropped != null) { - for (Map.Entry entry : dropped.entrySet()) { - log.info( - "[%s] : Dropped %s segments among %,d servers", - entry.getKey(), entry.getValue().get(), cluster.get(entry.getKey()).size() - ); - } - } + stats.forEachTieredStat( + "droppedCount", + (final String tier, final long count) -> { + log.info( + "[%s] : Dropped %s segments among %,d servers", + tier, count, cluster.getHistoricalsByTier(tier).size() + ); - emitTieredStats( - emitter, "segment/dropped/count", - dropped + emitTieredStat(emitter, "segment/dropped/count", tier, count); + } ); emitTieredStats( emitter, "segment/cost/raw", - stats.getPerTierStats().get("initialCost") + stats, "initialCost" ); emitTieredStats( emitter, "segment/cost/normalization", - stats.getPerTierStats().get("normalization") + stats, "normalization" ); emitTieredStats( emitter, "segment/moved/count", - stats.getPerTierStats().get("movedCount") + stats, "movedCount" ); emitTieredStats( emitter, "segment/deleted/count", - stats.getPerTierStats().get("deletedCount") + stats, "deletedCount" ); - Map normalized = stats.getPerTierStats().get("normalizedInitialCostTimesOneThousand"); - if (normalized != null) { - emitTieredStats( - emitter, "segment/cost/normalized", - Maps.transformEntries( - normalized, - new Maps.EntryTransformer() - { - @Override - public Number transformEntry(String key, AtomicLong value) - { - return value.doubleValue() / 1000d; - } - } - ) - ); - } - - Map unneeded = stats.getPerTierStats().get("unneededCount"); - if (unneeded != null) { - for (Map.Entry entry : unneeded.entrySet()) { - log.info( - "[%s] : Removed %s unneeded segments among %,d servers", - entry.getKey(), entry.getValue().get(), cluster.get(entry.getKey()).size() - ); - } - } + stats.forEachTieredStat( + "normalizedInitialCostTimesOneThousand", + (final String tier, final long count) -> { + emitTieredStat(emitter, "segment/cost/normalized", tier, count / 1000d); + } + ); - emitTieredStats( - emitter, "segment/unneeded/count", - stats.getPerTierStats().get("unneededCount") + stats.forEachTieredStat( + "unneededCount", + (final String tier, final long count) -> { + log.info( + "[%s] : Removed %s unneeded segments among %,d servers", + tier, count, cluster.getHistoricalsByTier(tier).size() + ); + emitTieredStat(emitter, "segment/unneeded/count", tier, count); + } ); emitter.emit( new ServiceMetricEvent.Builder().build( - "segment/overShadowed/count", stats.getGlobalStats().get("overShadowedCount") + "segment/overShadowed/count", + stats.getGlobalStat("overShadowedCount") ) ); - Map moved = stats.getPerTierStats().get("movedCount"); - if (moved != null) { - for (Map.Entry entry : moved.entrySet()) { - log.info( - "[%s] : Moved %,d segment(s)", - entry.getKey(), entry.getValue().get() - ); - } - } - final Map unmoved = stats.getPerTierStats().get("unmovedCount"); - if (unmoved != null) { - for(Map.Entry entry : unmoved.entrySet()) { - log.info( - "[%s] : Let alone %,d segment(s)", - entry.getKey(), entry.getValue().get() - ); - } - } + stats.forEachTieredStat( + "movedCount", + (final String tier, final long count) -> { + log.info("[%s] : Moved %,d segment(s)", tier, count); + } + ); + + stats.forEachTieredStat( + "unmovedCount", + (final String tier, final long count) -> { + log.info("[%s] : Let alone %,d segment(s)", tier, count); + } + ); + log.info("Load Queues:"); - for (MinMaxPriorityQueue serverHolders : cluster.getSortedServersByTier()) { + for (MinMaxPriorityQueue serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { ImmutableDruidServer server = serverHolder.getServer(); LoadQueuePeon queuePeon = serverHolder.getPeon(); log.info( "Server[%s, %s, %s] has %,d left to load, %,d left to drop, %,d bytes queued, %,d bytes served.", server.getName(), - server.getType(), + server.getType().toString(), server.getTier(), queuePeon.getSegmentsToLoad().size(), queuePeon.getSegmentsToDrop().size(), @@ -213,91 +197,92 @@ public Number transformEntry(String key, AtomicLong value) } // Emit coordinator metrics - final Set> peonEntries = params.getLoadManagementPeons().entrySet(); - for (Map.Entry entry : peonEntries) { - String serverName = entry.getKey(); - LoadQueuePeon queuePeon = entry.getValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/size", queuePeon.getLoadQueueSize() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/count", queuePeon.getSegmentsToLoad().size() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/dropQueue/count", queuePeon.getSegmentsToDrop().size() - ) - ); - } - for (Map.Entry entry : coordinator.getSegmentAvailability().entrySet()) { - String datasource = entry.getKey(); - Long count = entry.getValue().get(); - emitter.emit( + params + .getLoadManagementPeons() + .forEach( + (final String serverName, final LoadQueuePeon queuePeon) -> { + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/loadQueue/size", queuePeon.getLoadQueueSize() + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/loadQueue/count", queuePeon.getSegmentsToLoad().size() + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/dropQueue/count", queuePeon.getSegmentsToDrop().size() + ) + ); + } + ); + + coordinator.getSegmentAvailability().object2LongEntrySet().forEach( + (final Object2LongMap.Entry entry) -> { + final String dataSource = entry.getKey(); + final long count = entry.getLongValue(); + emitter.emit( new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, datasource).build( - "segment/unavailable/count", count + .setDimension(DruidMetrics.DATASOURCE, dataSource).build( + "segment/unavailable/count", count ) - ); - } - for (Map.Entry> entry : coordinator.getReplicationStatus().entrySet()) { - String tier = entry.getKey(); - CountingMap datasourceAvailabilities = entry.getValue(); - for (Map.Entry datasourceAvailability : datasourceAvailabilities.entrySet()) { - String datasource = datasourceAvailability.getKey(); - Long count = datasourceAvailability.getValue().get(); - emitter.emit( + ); + } + ); + + coordinator.getReplicationStatus().forEach( + (final String tier, final Object2LongMap status) -> { + for (final Object2LongMap.Entry entry : status.object2LongEntrySet()) { + final String dataSource = entry.getKey(); + final long count = entry.getLongValue(); + + emitter.emit( new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.TIER, tier) - .setDimension(DruidMetrics.DATASOURCE, datasource).build( - "segment/underReplicated/count", count + .setDimension(DruidMetrics.TIER, tier) + .setDimension(DruidMetrics.DATASOURCE, dataSource).build( + "segment/underReplicated/count", count ) - ); - } - } + ); + } + } + ); // Emit segment metrics - CountingMap segmentSizes = new CountingMap(); - CountingMap segmentCounts = new CountingMap(); - for (DruidDataSource dataSource : params.getDataSources()) { - for (DataSegment segment : dataSource.getSegments()) { - segmentSizes.add(dataSource.getName(), segment.getSize()); - segmentCounts.add(dataSource.getName(), 1L); - } - } - for (Map.Entry entry : segmentSizes.snapshot().entrySet()) { - String dataSource = entry.getKey(); - Long size = entry.getValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, dataSource).build( - "segment/size", size - ) - ); - } - for (Map.Entry entry : segmentCounts.snapshot().entrySet()) { - String dataSource = entry.getKey(); - Long count = entry.getValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, dataSource).build( - "segment/count", count - ) - ); - } + final Stream allSegments = params + .getDataSources() + .stream() + .flatMap((final DruidDataSource dataSource) -> dataSource.getSegments().stream()); + allSegments + .collect(Collectors.groupingBy(DataSegment::getDataSource)) + .forEach( + (final String name, final List segments) -> { + final long size = segments.stream().mapToLong(DataSegment::getSize).sum(); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, name).build( + "segment/size", size + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, name).build( + "segment/count", segments.size() + ) + ); + } + ); return params; } diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentMerger.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentMerger.java index ca2459f4707b..275758f4b2fa 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentMerger.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentMerger.java @@ -130,11 +130,11 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } } - log.info("Issued merge requests for %s segments", stats.getGlobalStats().get("mergedCount").get()); + log.info("Issued merge requests for %s segments", stats.getGlobalStat("mergedCount")); params.getEmitter().emit( new ServiceMetricEvent.Builder().build( - "coordinator/merge/count", stats.getGlobalStats().get("mergedCount") + "coordinator/merge/count", stats.getGlobalStat("mergedCount") ) ); diff --git a/server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java index 8a0e4a5679b6..89d85397e910 100644 --- a/server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java @@ -58,6 +58,7 @@ public String getType() return "loadByInterval"; } + @Override @JsonProperty public Map getTieredReplicants() { diff --git a/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java index c260ec23bbc9..3ce2dcb75936 100644 --- a/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java @@ -23,7 +23,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.emitter.EmittingLogger; - import io.druid.java.util.common.IAE; import io.druid.server.coordinator.BalancerStrategy; import io.druid.server.coordinator.CoordinatorStats; @@ -64,7 +63,7 @@ public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntim final int loadedReplicantsInTier = params.getSegmentReplicantLookup() .getLoadedReplicants(segment.getIdentifier(), tier); - final MinMaxPriorityQueue serverQueue = params.getDruidCluster().getServersByTier(tier); + final MinMaxPriorityQueue serverQueue = params.getDruidCluster().getHistoricalsByTier(tier); if (serverQueue == null) { log.makeAlert("Tier[%s] has no servers! Check your cluster configuration!", tier).emit(); continue; @@ -84,7 +83,7 @@ public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntim segment ); stats.accumulate(assignStats); - totalReplicantsInCluster += assignStats.getPerTierStats().get(ASSIGNED_COUNT).get(tier).get(); + totalReplicantsInCluster += assignStats.getTieredStat(ASSIGNED_COUNT, tier); } loadStatus.put(tier, expectedReplicantsInTier - loadedReplicantsInTier); @@ -176,8 +175,6 @@ private CoordinatorStats drop( } } - final ReplicationThrottler replicationManager = params.getReplicationManager(); - // Find all instances of this segment across tiers Map replicantsByTier = params.getSegmentReplicantLookup().getClusterTiers(segment.getIdentifier()); @@ -188,7 +185,7 @@ private CoordinatorStats drop( stats.addToTieredStat(DROPPED_COUNT, tier, 0); - MinMaxPriorityQueue serverQueue = params.getDruidCluster().get(tier); + MinMaxPriorityQueue serverQueue = params.getDruidCluster().getHistoricalsByTier(tier); if (serverQueue == null) { log.makeAlert("No holders found for tier[%s]", entry.getKey()).emit(); continue; diff --git a/server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java b/server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java index 3d2f46d0a154..52435f86cf5c 100644 --- a/server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java @@ -20,26 +20,34 @@ package io.druid.server.http; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.druid.server.coordinator.DruidCoordinator; import java.net.URL; +import java.util.Set; /** -*/ + */ public class CoordinatorRedirectInfo implements RedirectInfo { + private static final Set LOCAL_PATHS = ImmutableSet.of( + "/druid/coordinator/v1/leader", + "/druid/coordinator/v1/isLeader" + ); + private final DruidCoordinator coordinator; @Inject - public CoordinatorRedirectInfo(DruidCoordinator coordinator) { + public CoordinatorRedirectInfo(DruidCoordinator coordinator) + { this.coordinator = coordinator; } @Override public boolean doLocal(String requestURI) { - return coordinator.isLeader(); + return (requestURI != null && LOCAL_PATHS.contains(requestURI)) || coordinator.isLeader(); } @Override diff --git a/server/src/main/java/io/druid/server/http/CoordinatorResource.java b/server/src/main/java/io/druid/server/http/CoordinatorResource.java index 20f6805dae12..eea70723635e 100644 --- a/server/src/main/java/io/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorResource.java @@ -36,6 +36,7 @@ import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.util.Map; /** */ @@ -61,6 +62,20 @@ public Response getLeader() return Response.ok(coordinator.getCurrentLeader()).build(); } + @GET + @Path("/isLeader") + @Produces(MediaType.APPLICATION_JSON) + public Response isLeader() + { + final boolean leading = coordinator.isLeader(); + final Map response = ImmutableMap.of("leader", leading); + if (leading) { + return Response.ok(response).build(); + } else { + return Response.status(Response.Status.NOT_FOUND).entity(response).build(); + } + } + @GET @Path("/loadstatus") @Produces(MediaType.APPLICATION_JSON) @@ -147,16 +162,16 @@ public String apply(DataSegment segment) ) .put( "segmentsToDrop", Collections2.transform( - input.getSegmentsToDrop(), - new Function() - { - @Override - public String apply(DataSegment segment) - { - return segment.getIdentifier(); - } - } - ) + input.getSegmentsToDrop(), + new Function() + { + @Override + public String apply(DataSegment segment) + { + return segment.getIdentifier(); + } + } + ) ) .build(); } diff --git a/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java b/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java index 38d1725d3d88..dd6520eea6f4 100644 --- a/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java +++ b/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java @@ -19,10 +19,14 @@ package io.druid.server.http; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; +import com.google.common.net.HostAndPort; import com.google.inject.Inject; import io.druid.audit.AuditInfo; import io.druid.audit.AuditManager; @@ -32,6 +36,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.RE; import io.druid.java.util.common.logger.Logger; +import io.druid.query.lookup.LookupsState; import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.server.lookup.cache.LookupExtractorFactoryMapContainer; @@ -51,7 +56,12 @@ import javax.ws.rs.core.Response; import java.io.IOException; import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.Objects; /** * Contains information about lookups exposed through the coordinator @@ -284,4 +294,310 @@ public Response getSpecificTier( return Response.serverError().entity(ServletResourceUtils.sanitizeException(e)).build(); } } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/status") + public Response getAllLookupsStatus( + @QueryParam("detailed") boolean detailed + ) + { + try { + Map> configuredLookups = lookupCoordinatorManager.getKnownLookups(); + if (configuredLookups == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("No lookups found")) + .build(); + } + + Map> lookupsStateOnNodes = lookupCoordinatorManager.getLastKnownLookupsStateOnNodes(); + + Map> result = new HashMap<>(); + + for (Map.Entry> tierEntry : configuredLookups.entrySet()) { + String tier = tierEntry.getKey(); + Map lookupStatusMap = new HashMap<>(); + result.put(tier, lookupStatusMap); + Collection hosts = lookupCoordinatorManager.discoverNodesInTier(tier); + + for (Map.Entry lookupsEntry : tierEntry.getValue().entrySet()) { + lookupStatusMap.put( + lookupsEntry.getKey(), + getLookupStatus( + lookupsEntry.getKey(), + lookupsEntry.getValue(), + hosts, + lookupsStateOnNodes, + detailed + ) + ); + } + } + + return Response.ok(result).build(); + } catch (Exception ex) { + LOG.error(ex, "Error getting lookups status"); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/status/{tier}") + public Response getLookupStatusForTier( + @PathParam("tier") String tier, + @QueryParam("detailed") boolean detailed + ) + { + try { + Map> configuredLookups = lookupCoordinatorManager.getKnownLookups(); + if (configuredLookups == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("No lookups found")) + .build(); + } + + Map tierLookups = configuredLookups.get(tier); + if (tierLookups == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("No lookups found for tier [%s].", tier)) + .build(); + } + + + Map lookupStatusMap = new HashMap<>(); + Collection hosts = lookupCoordinatorManager.discoverNodesInTier(tier); + + Map> lookupsStateOnNodes = lookupCoordinatorManager.getLastKnownLookupsStateOnNodes(); + + for (Map.Entry lookupsEntry : tierLookups.entrySet()) { + lookupStatusMap.put( + lookupsEntry.getKey(), + getLookupStatus(lookupsEntry.getKey(), lookupsEntry.getValue(), hosts, lookupsStateOnNodes, detailed) + ); + } + + return Response.ok(lookupStatusMap).build(); + } catch (Exception ex) { + LOG.error(ex, "Error getting lookups status for tier [%s].", tier); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/status/{tier}/{lookup}") + public Response getSpecificLookupStatus( + @PathParam("tier") String tier, + @PathParam("lookup") String lookup, + @QueryParam("detailed") boolean detailed + ) + { + try { + Map> configuredLookups = lookupCoordinatorManager.getKnownLookups(); + if (configuredLookups == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("No lookups found")) + .build(); + } + + Map tierLookups = configuredLookups.get(tier); + if (tierLookups == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("No lookups found for tier [%s].", tier)) + .build(); + } + + LookupExtractorFactoryMapContainer lookupDef = tierLookups.get(lookup); + if (lookupDef == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("Lookup [%s] not found for tier [%s].", lookup, tier)) + .build(); + } + + return Response.ok( + getLookupStatus( + lookup, + lookupDef, + lookupCoordinatorManager.discoverNodesInTier(tier), + lookupCoordinatorManager.getLastKnownLookupsStateOnNodes(), + detailed + ) + ).build(); + } catch (Exception ex) { + LOG.error(ex, "Error getting lookups status for tier [%s] and lookup [%s].", tier, lookup); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + } + + @VisibleForTesting + LookupStatus getLookupStatus( + String lookupName, + LookupExtractorFactoryMapContainer lookupDef, + Collection nodes, + Map> lastKnownLookupsState, + boolean detailed + ) + { + boolean isReady = true; + List pendingHosts = detailed ? new ArrayList<>() : null; + + for (HostAndPort node : nodes) { + LookupsState hostState = lastKnownLookupsState.get(node); + LookupExtractorFactoryMapContainer loadedOnHost = hostState != null + ? hostState.getCurrent().get(lookupName) + : null; + if (loadedOnHost == null || lookupDef.replaces(loadedOnHost)) { + isReady = false; + if (detailed) { + pendingHosts.add(node); + } else { + break; + } + } + } + + return new LookupStatus(isReady, pendingHosts); + } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/nodeStatus") + public Response getAllNodesStatus( + @QueryParam("discover") boolean discover + ) + { + try { + Collection tiers = null; + if (discover) { + tiers = lookupCoordinatorManager.discoverTiers(); + } else { + Map> configuredLookups = lookupCoordinatorManager.getKnownLookups(); + if (configuredLookups == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("No lookups configured.")) + .build(); + } + tiers = configuredLookups.keySet(); + } + + Map> lookupsStateOnHosts = lookupCoordinatorManager.getLastKnownLookupsStateOnNodes(); + + Map>> result = new HashMap<>(); + + for (String tier : tiers) { + Map> tierNodesStatus = new HashMap<>(); + result.put(tier, tierNodesStatus); + + Collection nodes = lookupCoordinatorManager.discoverNodesInTier(tier); + + for (HostAndPort node : nodes) { + LookupsState lookupsState = lookupsStateOnHosts.get(node); + if (lookupsState == null) { + tierNodesStatus.put(node, new LookupsState<>(null,null,null)); + } else { + tierNodesStatus.put(node, lookupsState); + } + } + } + + return Response.ok(result).build(); + } catch (Exception ex) { + LOG.error(ex, "Error getting node status."); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/nodeStatus/{tier}") + public Response getNodesStatusInTier( + @PathParam("tier") String tier + ) + { + try { + Map> lookupsStateOnHosts = lookupCoordinatorManager.getLastKnownLookupsStateOnNodes(); + + Map> tierNodesStatus = new HashMap<>(); + + Collection nodes = lookupCoordinatorManager.discoverNodesInTier(tier); + + for (HostAndPort node : nodes) { + LookupsState lookupsState = lookupsStateOnHosts.get(node); + if (lookupsState == null) { + tierNodesStatus.put(node, new LookupsState<>(null,null,null)); + } else { + tierNodesStatus.put(node, lookupsState); + } + } + + return Response.ok(tierNodesStatus).build(); + } catch (Exception ex) { + LOG.error(ex, "Error getting node status for tier [%s].", tier); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/nodeStatus/{tier}/{hostAndPort}") + public Response getSpecificNodeStatus( + @PathParam("tier") String tier, + @PathParam("hostAndPort") HostAndPort hostAndPort + ) + { + try { + Map> lookupsStateOnHosts = lookupCoordinatorManager.getLastKnownLookupsStateOnNodes(); + + LookupsState lookupsState = lookupsStateOnHosts.get(hostAndPort); + if (lookupsState == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ServletResourceUtils.jsonize("Node [%s] status is unknown.", hostAndPort)) + .build(); + } else { + return Response.ok(lookupsState).build(); + } + + } catch (Exception ex) { + LOG.error(ex, "Error getting node status for [%s].", hostAndPort); + return Response.serverError().entity(ServletResourceUtils.sanitizeException(ex)).build(); + } + } + + @VisibleForTesting + static class LookupStatus + { + @JsonProperty + private boolean loaded; + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + private List pendingNodes; + + public LookupStatus(boolean loaded, List pendingHosts) + { + this.loaded = loaded; + this.pendingNodes = pendingHosts; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LookupStatus that = (LookupStatus) o; + return Objects.equals(loaded, that.loaded) && + Objects.equals(pendingNodes, that.pendingNodes); + } + + @Override + public int hashCode() + { + return Objects.hash(loaded, pendingNodes); + } + } } diff --git a/server/src/main/java/io/druid/server/http/SegmentListerResource.java b/server/src/main/java/io/druid/server/http/SegmentListerResource.java new file mode 100644 index 000000000000..c24ec4c60a1c --- /dev/null +++ b/server/src/main/java/io/druid/server/http/SegmentListerResource.java @@ -0,0 +1,229 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import com.metamx.emitter.EmittingLogger; +import com.sun.jersey.spi.container.ResourceFilters; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Smile; +import io.druid.server.coordination.BatchDataSegmentAnnouncer; +import io.druid.server.coordination.SegmentChangeRequestHistory; +import io.druid.server.coordination.SegmentChangeRequestsSnapshot; +import io.druid.server.http.security.StateResourceFilter; +import io.druid.server.security.AuthConfig; + +import javax.annotation.Nullable; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import java.io.IOException; + +/** + */ +@Path("/druid-internal/v1/segments/") +@ResourceFilters(StateResourceFilter.class) +public class SegmentListerResource +{ + protected static final EmittingLogger log = new EmittingLogger(SegmentListerResource.class); + + protected final ObjectMapper jsonMapper; + protected final ObjectMapper smileMapper; + protected final AuthConfig authConfig; + private final BatchDataSegmentAnnouncer announcer; + + @Inject + public SegmentListerResource( + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper, + AuthConfig authConfig, + @Nullable BatchDataSegmentAnnouncer announcer + ) + { + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + this.authConfig = authConfig; + this.announcer = announcer; + } + + /** + * This endpoint is used by HttpServerInventoryView to keep an up-to-date list of segments served by + * historical/realtime nodes. + * + * This endpoint lists segments served by this server and can also incrementally provide the segments added/dropped + * since last response. + * + * Here is how, this is used. + * + * (1) Client sends first request /druid/internal/v1/segments?counter=-1&timeout= + * Server responds with list of segments currently served and a pair. + * + * (2) Client sends subsequent requests /druid/internal/v1/segments?counter=&hash=&timeout= + * Where values are used from the last response. Server responds with list of segment updates + * since given counter. + * + * This endpoint makes the client wait till either there is some segment update or given timeout elapses. + * + * So, clients keep on sending next request immediately after receiving the response in order to keep the list + * of segments served by this server up-to-date. + * + * @param counter counter received in last response. + * @param hash hash received in last response. + * @param timeout after which response is sent even if there are no new segment updates. + * @param req + * @throws IOException + */ + @GET + @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + public void getSegments( + @QueryParam("counter") long counter, + @QueryParam("hash") long hash, + @QueryParam("timeout") long timeout, + @Context final HttpServletRequest req + ) throws IOException + { + if (announcer == null) { + sendErrorResponse(req, HttpServletResponse.SC_NOT_FOUND, "announcer is not available."); + return; + } + + if (timeout <= 0) { + sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "timeout must be positive."); + return; + } + + final ResponseContext context = createContext(req.getHeader("Accept")); + final ListenableFuture future = announcer.getSegmentChangesSince( + new SegmentChangeRequestHistory.Counter( + counter, + hash + ) + ); + + final AsyncContext asyncContext = req.startAsync(); + + asyncContext.addListener( + new AsyncListener() + { + @Override + public void onComplete(AsyncEvent event) throws IOException + { + } + + @Override + public void onTimeout(AsyncEvent event) throws IOException + { + + // HTTP 204 NO_CONTENT is sent to the client. + future.cancel(true); + event.getAsyncContext().complete(); + } + + @Override + public void onError(AsyncEvent event) throws IOException + { + } + + @Override + public void onStartAsync(AsyncEvent event) throws IOException + { + } + } + ); + + Futures.addCallback( + future, + new FutureCallback() + { + @Override + public void onSuccess(SegmentChangeRequestsSnapshot result) + { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(HttpServletResponse.SC_OK); + context.inputMapper.writeValue(asyncContext.getResponse().getOutputStream(), result); + asyncContext.complete(); + } + catch (Exception ex) { + log.debug(ex, "Request timed out or closed already."); + } + } + + @Override + public void onFailure(Throwable th) + { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + if (th instanceof IllegalArgumentException) { + response.sendError(HttpServletResponse.SC_BAD_REQUEST, th.getMessage()); + } else { + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, th.getMessage()); + } + asyncContext.complete(); + } + catch (Exception ex) { + log.debug(ex, "Request timed out or closed already."); + } + } + } + ); + + asyncContext.setTimeout(timeout); + } + + private void sendErrorResponse(HttpServletRequest req, int code, String error) throws IOException + { + AsyncContext asyncContext = req.startAsync(); + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.sendError(code, error); + asyncContext.complete(); + } + + private ResponseContext createContext(String requestType) + { + boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(requestType); + return new ResponseContext(isSmile ? smileMapper : jsonMapper); + } + + private static class ResponseContext + { + private final ObjectMapper inputMapper; + + ResponseContext(ObjectMapper inputMapper) + { + this.inputMapper = inputMapper; + } + } +} diff --git a/server/src/main/java/io/druid/server/http/ServersResource.java b/server/src/main/java/io/druid/server/http/ServersResource.java index 70308eb8ebb0..baa2464d4098 100644 --- a/server/src/main/java/io/druid/server/http/ServersResource.java +++ b/server/src/main/java/io/druid/server/http/ServersResource.java @@ -51,7 +51,7 @@ private static Map makeSimpleServer(DruidServer input) return new ImmutableMap.Builder() .put("host", input.getHost()) .put("tier", input.getTier()) - .put("type", input.getType()) + .put("type", input.getType().toString()) .put("priority", input.getPriority()) .put("currSize", input.getCurrSize()) .put("maxSize", input.getMaxSize()) @@ -63,7 +63,7 @@ private static Map makeFullServer(DruidServer input) return new ImmutableMap.Builder() .put("host", input.getHost()) .put("maxSize", input.getMaxSize()) - .put("type", input.getType()) + .put("type", input.getType().toString()) .put("tier", input.getTier()) .put("priority", input.getPriority()) .put("segments", input.getSegments()) diff --git a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java index 960b3c123ccb..39d825130c33 100644 --- a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java @@ -82,6 +82,7 @@ public ContainerRequest filter(ContainerRequest request) return request; } + @Override public boolean isApplicable(String requestPath) { return requestPath.startsWith("druid/broker/v1") || diff --git a/server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java b/server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java index ce7e95521964..c9c6f1f1d982 100644 --- a/server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java @@ -45,6 +45,9 @@ public class BatchDataSegmentAnnouncerConfig @JsonProperty private boolean skipDimensionsAndMetrics = false; + @JsonProperty + private boolean skipSegmentAnnouncementOnZk = false; + public int getSegmentsPerNode() { return segmentsPerNode; @@ -65,4 +68,8 @@ public boolean isSkipDimensionsAndMetrics() return skipDimensionsAndMetrics; } + public boolean isSkipSegmentAnnouncementOnZk() + { + return skipSegmentAnnouncementOnZk; + } } diff --git a/server/src/main/java/io/druid/server/initialization/ServerConfig.java b/server/src/main/java/io/druid/server/initialization/ServerConfig.java index 560975abe186..97ba7b888b82 100644 --- a/server/src/main/java/io/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ServerConfig.java @@ -41,6 +41,10 @@ public class ServerConfig @Min(0) private long defaultQueryTimeout = 300_000; // 5 minutes + @JsonProperty + @Min(1) + private long maxScatterGatherBytes = Long.MAX_VALUE; + public int getNumThreads() { return numThreads; @@ -56,6 +60,11 @@ public long getDefaultQueryTimeout() return defaultQueryTimeout; } + public long getMaxScatterGatherBytes() + { + return maxScatterGatherBytes; + } + @Override public String toString() { @@ -63,6 +72,7 @@ public String toString() "numThreads=" + numThreads + ", maxIdleTime=" + maxIdleTime + ", defaultQueryTimeout=" + defaultQueryTimeout + + ", maxScatterGatherBytes=" + maxScatterGatherBytes + '}'; } } diff --git a/server/src/main/java/io/druid/server/initialization/jetty/ResponseHeaderFilterHolder.java b/server/src/main/java/io/druid/server/initialization/jetty/ResponseHeaderFilterHolder.java index d9333586c0f9..4883e42bf48c 100644 --- a/server/src/main/java/io/druid/server/initialization/jetty/ResponseHeaderFilterHolder.java +++ b/server/src/main/java/io/druid/server/initialization/jetty/ResponseHeaderFilterHolder.java @@ -75,7 +75,7 @@ public EnumSet getDispatcherType() return null; } - private class ResponseHeaderFilter implements Filter + private static class ResponseHeaderFilter implements Filter { private volatile FilterConfig config; diff --git a/server/src/main/java/io/druid/server/log/FileRequestLogger.java b/server/src/main/java/io/druid/server/log/FileRequestLogger.java index 88a61bd31224..4f3fefbc9e8d 100644 --- a/server/src/main/java/io/druid/server/log/FileRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/FileRequestLogger.java @@ -73,11 +73,11 @@ public void start() fileWriter = getFileWriter(); } long nextDay = currentDay.plusDays(1).getMillis(); - Duration delay = new Duration(nextDay - new DateTime().getMillis()); + Duration initialDelay = new Duration(nextDay - new DateTime().getMillis()); ScheduledExecutors.scheduleWithFixedDelay( exec, - delay, + initialDelay, Duration.standardDays(1), new Callable() { diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index 7b9409d2449c..fd36a8ea17ca 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -116,7 +116,7 @@ public class LookupCoordinatorManager // management loop, then they get discarded automatically. @VisibleForTesting final AtomicReference>> knownOldState = - new AtomicReference<>(); + new AtomicReference<>(ImmutableMap.of()); // Updated by config watching service private AtomicReference>> lookupMapConfigRef; @@ -275,6 +275,7 @@ public boolean deleteLookup(final String tier, final String lookup, AuditInfo au public Collection discoverTiers() { try { + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); return listenerDiscoverer.discoverChildren(LookupCoordinatorManager.LOOKUP_LISTEN_ANNOUNCE_KEY); } catch (IOException e) { @@ -282,6 +283,22 @@ public Collection discoverTiers() } } + public Collection discoverNodesInTier(String tier) + { + try { + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); + return listenerDiscoverer.getNodes(LookupModule.getTierListenerPath(tier)); + } catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public Map> getLastKnownLookupsStateOnNodes() + { + Preconditions.checkState(lifecycleLock.awaitStarted(5, TimeUnit.SECONDS), "not started"); + return knownOldState.get(); + } + /** * Try to find a lookupName spec for the specified lookupName. * diff --git a/server/src/main/java/io/druid/server/metrics/HistoricalMetricsMonitor.java b/server/src/main/java/io/druid/server/metrics/HistoricalMetricsMonitor.java index cb3bb6b7f063..f6bd6e2e6059 100644 --- a/server/src/main/java/io/druid/server/metrics/HistoricalMetricsMonitor.java +++ b/server/src/main/java/io/druid/server/metrics/HistoricalMetricsMonitor.java @@ -24,29 +24,30 @@ import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.AbstractMonitor; import io.druid.client.DruidServerConfig; -import io.druid.java.util.common.collect.CountingMap; import io.druid.query.DruidMetrics; -import io.druid.server.coordination.ServerManager; +import io.druid.server.SegmentManager; import io.druid.server.coordination.ZkCoordinator; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import java.util.Map; public class HistoricalMetricsMonitor extends AbstractMonitor { private final DruidServerConfig serverConfig; - private final ServerManager serverManager; + private final SegmentManager segmentManager; private final ZkCoordinator zkCoordinator; @Inject public HistoricalMetricsMonitor( DruidServerConfig serverConfig, - ServerManager serverManager, + SegmentManager segmentManager, ZkCoordinator zkCoordinator ) { this.serverConfig = serverConfig; - this.serverManager = serverManager; + this.segmentManager = segmentManager; this.zkCoordinator = zkCoordinator; } @@ -55,15 +56,16 @@ public boolean doMonitor(ServiceEmitter emitter) { emitter.emit(new ServiceMetricEvent.Builder().build("segment/max", serverConfig.getMaxSize())); - final CountingMap pendingDeleteSizes = new CountingMap(); + final Object2LongOpenHashMap pendingDeleteSizes = new Object2LongOpenHashMap<>(); for (DataSegment segment : zkCoordinator.getPendingDeleteSnapshot()) { - pendingDeleteSizes.add(segment.getDataSource(), segment.getSize()); + pendingDeleteSizes.addTo(segment.getDataSource(), segment.getSize()); } - for (Map.Entry entry : pendingDeleteSizes.entrySet()) { + for (final Object2LongMap.Entry entry : pendingDeleteSizes.object2LongEntrySet()) { + final String dataSource = entry.getKey(); - final long pendingDeleteSize = entry.getValue(); + final long pendingDeleteSize = entry.getLongValue(); emitter.emit( new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, dataSource) @@ -73,7 +75,7 @@ public boolean doMonitor(ServiceEmitter emitter) ); } - for (Map.Entry entry : serverManager.getDataSourceSizes().entrySet()) { + for (Map.Entry entry : segmentManager.getDataSourceSizes().entrySet()) { String dataSource = entry.getKey(); long used = entry.getValue(); @@ -88,7 +90,7 @@ public boolean doMonitor(ServiceEmitter emitter) emitter.emit(builder.build("segment/usedPercent", usedPercent)); } - for (Map.Entry entry : serverManager.getDataSourceCounts().entrySet()) { + for (Map.Entry entry : segmentManager.getDataSourceCounts().entrySet()) { String dataSource = entry.getKey(); long count = entry.getValue(); final ServiceMetricEvent.Builder builder = diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java index 85af3764ab9b..72f0cd8c9e50 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java @@ -122,6 +122,7 @@ public String getDefaultServiceName() return tierConfig.getDefaultBrokerServiceName(); } + @Override public Pair select(final Query query) { synchronized (lock) { diff --git a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java index 7f019e351935..593dad190d00 100644 --- a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java @@ -29,6 +29,9 @@ import java.util.List; import java.util.Map; +/** + * An extendable linear shard spec. {@link #partitionNum} represents an unique id of a partition. + */ public class LinearShardSpec implements ShardSpec { private int partitionNum; diff --git a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java index e46e5e0bb875..0aca040b8ded 100644 --- a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java @@ -30,6 +30,13 @@ import java.util.List; import java.util.Map; +/** + * An extendable linear shard spec containing the information of core partitions. This class contains two variables of + * {@link #partitionNum} and {@link #partitions}, which represent the unique id of a partition and the number of core + * partitions, respectively. {@link #partitions} simply indicates that the atomic update is regarded as completed when + * {@link #partitions} partitions are successfully updated, and {@link #partitionNum} can go beyond it when some types + * of index tasks are trying to append to existing partitions. + */ public class NumberedShardSpec implements ShardSpec { @JsonIgnore diff --git a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java index 41fe73fbe290..8cbefe40f8a1 100644 --- a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java @@ -92,6 +92,7 @@ public void setEnd(String end) this.end = end; } + @Override @JsonProperty("partitionNum") public int getPartitionNum() { diff --git a/server/src/main/resources/static/old-console/js/init-0.0.2.js b/server/src/main/resources/static/old-console/js/init-0.0.2.js index c89ac19ce13a..fc6b07f01c1b 100644 --- a/server/src/main/resources/static/old-console/js/init-0.0.2.js +++ b/server/src/main/resources/static/old-console/js/init-0.0.2.js @@ -53,8 +53,12 @@ $(document).ready(function() { return ret; }()); var avg = serverTable.getColTotal('Server percentUsed') / serverTable.getNumRows(); - $('#avg_server_metric').html('Average Server Percent Used: ' + avg + '%'); - + if (!isNaN(avg)) { + $('#avg_server_metric').html('Average Server Percent Used: ' + avg + '%'); + }else{ + $('.loading').html('Server is still starting...Please try after few minutes.'); + $('.loading').show() + } serverTable.toHTMLTable($('#servers')); segmentTable.toHTMLTable($('#segments')); } diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index 6049837a4b54..f2d5bcb12e5c 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -39,6 +39,7 @@ import io.druid.query.QueryWatcher; import io.druid.query.TableDataSource; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; @@ -98,7 +99,7 @@ public void testSingleServerAddedRemovedSegment() throws Exception "localhost:1234", "localhost:1234", 10000000L, - "historical", + ServerType.HISTORICAL, "default_tier", 0 ); @@ -164,7 +165,7 @@ public DruidServer apply(String input) input, input, 10000000L, - "historical", + ServerType.HISTORICAL, "default_tier", 0 ); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java index 88948d718f63..6aa88d4b6eda 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java @@ -35,10 +35,12 @@ import io.druid.query.QueryRunner; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.SingleElementPartitionChunk; +import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; @@ -46,17 +48,18 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeMap; import java.util.concurrent.Executor; /** */ -public class CachingClusteredClientFunctionalityTest { +public class CachingClusteredClientFunctionalityTest +{ public CachingClusteredClient client; @@ -74,17 +77,22 @@ public void setUp() throws Exception } @Test - public void testUncoveredInterval() throws Exception { + public void testUncoveredInterval() throws Exception + { addToTimeline(new Interval("2015-01-02/2015-01-03"), "1"); addToTimeline(new Interval("2015-01-04/2015-01-05"), "1"); addToTimeline(new Interval("2015-02-04/2015-02-05"), "1"); final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource("test") - .intervals("2015-01-02/2015-01-03") - .granularity("day") - .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) - .context(ImmutableMap.of("uncoveredIntervalsLimit", 3)); + .dataSource("test") + .intervals("2015-01-02/2015-01-03") + .granularity("day") + .aggregators(Arrays.asList(new CountAggregatorFactory( + "rows"))) + .context(ImmutableMap.of( + "uncoveredIntervalsLimit", + 3 + )); Map responseContext = new HashMap<>(); client.run(builder.build(), responseContext); @@ -131,7 +139,8 @@ public void testUncoveredInterval() throws Exception { assertUncovered(responseContext, true, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04", "2015-01-05/2015-02-04"); } - private void assertUncovered(Map context, boolean uncoveredIntervalsOverflowed, String... intervals) { + private void assertUncovered(Map context, boolean uncoveredIntervalsOverflowed, String... intervals) + { List expectedList = Lists.newArrayListWithExpectedSize(intervals.length); for (String interval : intervals) { expectedList.add(new Interval(interval)); @@ -140,28 +149,49 @@ private void assertUncovered(Map context, boolean uncoveredInter Assert.assertEquals(uncoveredIntervalsOverflowed, context.get("uncoveredIntervalsOverflowed")); } - private void addToTimeline(Interval interval, String version) { + private void addToTimeline(Interval interval, String version) + { timeline.add(interval, version, new SingleElementPartitionChunk<>( new ServerSelector( DataSegment.builder() - .dataSource("test") - .interval(interval) - .version(version) - .shardSpec(NoneShardSpec.instance()) - .build(), - new TierSelectorStrategy() { + .dataSource("test") + .interval(interval) + .version(version) + .shardSpec(NoneShardSpec.instance()) + .build(), + new TierSelectorStrategy() + { @Override - public Comparator getComparator() { + public Comparator getComparator() + { return Ordering.natural(); } @Override - public QueryableDruidServer pick(TreeMap> prioritizedServers, DataSegment segment) { + public QueryableDruidServer pick( + Int2ObjectRBTreeMap> prioritizedServers, DataSegment segment + ) + { return new QueryableDruidServer( - new DruidServer("localhost", "localhost", 100, "historical", "a", 10), + new DruidServer("localhost", "localhost", 100, ServerType.HISTORICAL, "a", 10), EasyMock.createNiceMock(DirectDruidClient.class) ); } + + @Override + public List pick( + Int2ObjectRBTreeMap> prioritizedServers, + DataSegment segment, + int numServersToPick + ) + { + return Collections.singletonList( + new QueryableDruidServer( + new DruidServer("localhost", "localhost", 100, ServerType.HISTORICAL, "a", 10), + EasyMock.createNiceMock(DirectDruidClient.class) + ) + ); + } } ) )); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index b81f1ee1b9a8..11482fb898c0 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -71,6 +71,7 @@ import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; @@ -121,6 +122,7 @@ import io.druid.query.topn.TopNQueryQueryToolChest; import io.druid.query.topn.TopNResultValue; import io.druid.segment.TestHelper; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.NoneShardSpec; @@ -327,11 +329,11 @@ public void setUp() throws Exception client = makeClient(MoreExecutors.sameThreadExecutor()); servers = new DruidServer[]{ - new DruidServer("test1", "test1", 10, "historical", "bye", 0), - new DruidServer("test2", "test2", 10, "historical", "bye", 0), - new DruidServer("test3", "test3", 10, "historical", "bye", 0), - new DruidServer("test4", "test4", 10, "historical", "bye", 0), - new DruidServer("test5", "test5", 10, "historical", "bye", 0) + new DruidServer("test1", "test1", 10, ServerType.HISTORICAL, "bye", 0), + new DruidServer("test2", "test2", 10, ServerType.HISTORICAL, "bye", 0), + new DruidServer("test3", "test3", 10, ServerType.HISTORICAL, "bye", 0), + new DruidServer("test4", "test4", 10, ServerType.HISTORICAL, "bye", 0), + new DruidServer("test5", "test5", 10, ServerType.HISTORICAL, "bye", 0) }; } @@ -398,6 +400,7 @@ public void onFailure(Throwable t) (ListenableFuture) delegate.submit((Runnable) task); } + @SuppressWarnings("ParameterPackage") @Override public ListenableFuture submit(Callable task) { @@ -1781,7 +1784,7 @@ client, new TimeseriesQueryQueryToolChest( timeline.add(interval2, "v", new StringPartitionChunk<>("d", null, 5, selector5)); timeline.add(interval3, "v", new StringPartitionChunk<>(null, null, 6, selector6)); - final Capture capture = Capture.newInstance(); + final Capture capture = Capture.newInstance(); final Capture> contextCap = Capture.newInstance(); QueryRunner mockRunner = EasyMock.createNiceMock(QueryRunner.class); @@ -1801,12 +1804,9 @@ client, new TimeseriesQueryQueryToolChest( descriptors.add(new SegmentDescriptor(interval3, "v", 6)); MultipleSpecificSegmentSpec expected = new MultipleSpecificSegmentSpec(descriptors); - Sequences.toList(runner.run( - query, - context - ), Lists.newArrayList()); + Sequences.toList(runner.run(QueryPlus.wrap(query), context), Lists.newArrayList()); - Assert.assertEquals(expected, capture.getValue().getQuerySegmentSpec()); + Assert.assertEquals(expected, ((TimeseriesQuery) capture.getValue().getQuery()).getQuerySegmentSpec()); } private ServerSelector makeMockSingleDimensionSelector( @@ -1923,7 +1923,7 @@ public void testQueryCachingWithFilter( .andReturn(expectations.getQueryRunner()) .times(0, 1); - final Capture capture = new Capture(); + final Capture capture = new Capture(); final Capture context = new Capture(); QueryRunner queryable = expectations.getQueryRunner(); @@ -1940,7 +1940,7 @@ public void testQueryCachingWithFilter( @Override public Sequence answer() throws Throwable { - return toFilteredQueryableTimeseriesResults((TimeseriesQuery)capture.getValue(), segmentIds, queryIntervals, results); + return toFilteredQueryableTimeseriesResults((TimeseriesQuery)capture.getValue().getQuery(), segmentIds, queryIntervals, results); } }) .times(0, 1); @@ -1965,10 +1965,12 @@ public void run() TestHelper.assertExpectedResults( expected, runner.run( - query.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec( - ImmutableList.of( - actualQueryInterval + QueryPlus.wrap( + query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec( + ImmutableList.of( + actualQueryInterval + ) ) ) ), @@ -2062,7 +2064,7 @@ public void testQueryCaching( .andReturn(expectations.getQueryRunner()) .once(); - final Capture capture = new Capture(); + final Capture capture = new Capture(); final Capture context = new Capture(); queryCaptures.add(capture); QueryRunner queryable = expectations.getQueryRunner(); @@ -2210,7 +2212,8 @@ public Iterable>> apply(@Nullable Integer input) // make sure all the queries were sent down as 'bySegment' for (Capture queryCapture : queryCaptures) { - Query capturedQuery = (Query) queryCapture.getValue(); + QueryPlus capturedQueryPlus = (QueryPlus) queryCapture.getValue(); + Query capturedQuery = capturedQueryPlus.getQuery(); if (expectBySegment) { Assert.assertEquals(true, capturedQuery.getContextValue("bySegment")); } else { diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index af57a5e97acb..d2e2d365c911 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -39,6 +39,7 @@ import io.druid.query.CacheStrategy; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; @@ -269,7 +270,7 @@ public void doMonitor(ServiceEmitter emitter) new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return resultSeq; } @@ -362,7 +363,7 @@ private void testUseCache( new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { return Sequences.empty(); } diff --git a/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java index 91143f8920bb..0ba1ac3ba392 100644 --- a/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java +++ b/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java @@ -32,6 +32,7 @@ import io.druid.java.util.common.Pair; import io.druid.query.TableDataSource; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; @@ -60,7 +61,7 @@ public class CoordinatorServerViewTest extends CuratorTestBase private CountDownLatch segmentAddedLatch; private CountDownLatch segmentRemovedLatch; - private ServerInventoryView baseView; + private BatchServerInventoryView baseView; private CoordinatorServerView overlordServerView; public CoordinatorServerViewTest() @@ -91,7 +92,7 @@ public void testSingleServerAddedRemovedSegment() throws Exception "localhost:1234", "localhost:1234", 10000000L, - "historical", + ServerType.HISTORICAL, "default_tier", 0 ); @@ -158,7 +159,7 @@ public DruidServer apply(String input) input, input, 10000000L, - "historical", + ServerType.HISTORICAL, "default_tier", 0 ); diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index c9ebab8eae8d..28130c85df74 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -42,6 +42,7 @@ import io.druid.query.ReflectionQueryToolChestWarehouse; import io.druid.query.Result; import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.server.coordination.ServerType; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -51,6 +52,7 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.jboss.netty.handler.timeout.ReadTimeoutException; import org.joda.time.DateTime; +import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -60,9 +62,20 @@ import java.net.URL; import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; public class DirectDruidClientTest { + private final Map defaultContext; + + public DirectDruidClientTest() + { + defaultContext = new HashMap<>(); + defaultContext.put(DirectDruidClient.QUERY_FAIL_TIME, Long.MAX_VALUE); + defaultContext.put(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED, new AtomicLong()); + } + @Test public void testRun() throws Exception { @@ -74,7 +87,8 @@ public void testRun() throws Exception EasyMock.expect( httpClient.go( EasyMock.capture(capturedRequest), - EasyMock.anyObject() + EasyMock.anyObject(), + EasyMock.anyObject(Duration.class) ) ) .andReturn(futureResult) @@ -84,7 +98,8 @@ public void testRun() throws Exception EasyMock.expect( httpClient.go( EasyMock.capture(capturedRequest), - EasyMock.anyObject() + EasyMock.anyObject(), + EasyMock.anyObject(Duration.class) ) ) .andReturn(futureException) @@ -93,7 +108,8 @@ public void testRun() throws Exception EasyMock.expect( httpClient.go( EasyMock.capture(capturedRequest), - EasyMock.anyObject() + EasyMock.anyObject(), + EasyMock.anyObject(Duration.class) ) ) .andReturn(SettableFuture.create()) @@ -134,34 +150,34 @@ public void testRun() throws Exception ); QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client1 ); serverSelector.addServerAndUpdateSegment(queryableDruidServer1, serverSelector.getSegment()); QueryableDruidServer queryableDruidServer2 = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client2 ); serverSelector.addServerAndUpdateSegment(queryableDruidServer2, serverSelector.getSegment()); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - HashMap context = Maps.newHashMap(); - Sequence s1 = client1.run(query, context); + + Sequence s1 = client1.run(query, defaultContext); Assert.assertTrue(capturedRequest.hasCaptured()); Assert.assertEquals(url, capturedRequest.getValue().getUrl()); Assert.assertEquals(HttpMethod.POST, capturedRequest.getValue().getMethod()); Assert.assertEquals(1, client1.getNumOpenConnections()); // simulate read timeout - Sequence s2 = client1.run(query, context); + Sequence s2 = client1.run(query, defaultContext); Assert.assertEquals(2, client1.getNumOpenConnections()); futureException.setException(new ReadTimeoutException()); Assert.assertEquals(1, client1.getNumOpenConnections()); // subsequent connections should work - Sequence s3 = client1.run(query, context); - Sequence s4 = client1.run(query, context); - Sequence s5 = client1.run(query, context); + Sequence s3 = client1.run(query, defaultContext); + Sequence s4 = client1.run(query, defaultContext); + Sequence s5 = client1.run(query, defaultContext); Assert.assertTrue(client1.getNumOpenConnections() == 4); @@ -172,8 +188,8 @@ public void testRun() throws Exception Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp()); Assert.assertEquals(3, client1.getNumOpenConnections()); - client2.run(query, context); - client2.run(query, context); + client2.run(query, defaultContext); + client2.run(query, defaultContext); Assert.assertTrue(client2.getNumOpenConnections() == 2); @@ -194,7 +210,8 @@ public void testCancel() throws Exception EasyMock.expect( httpClient.go( EasyMock.capture(capturedRequest), - EasyMock.anyObject() + EasyMock.anyObject(), + EasyMock.anyObject(Duration.class) ) ) .andReturn(cancelledFuture) @@ -203,7 +220,8 @@ public void testCancel() throws Exception EasyMock.expect( httpClient.go( EasyMock.capture(capturedRequest), - EasyMock.anyObject() + EasyMock.anyObject(), + EasyMock.anyObject(Duration.class) ) ) .andReturn(cancellationFuture) @@ -236,15 +254,14 @@ public void testCancel() throws Exception ); QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client1 ); serverSelector.addServerAndUpdateSegment(queryableDruidServer1, serverSelector.getSegment()); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - HashMap context = Maps.newHashMap(); cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); - Sequence results = client1.run(query, context); + Sequence results = client1.run(query, defaultContext); Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod()); Assert.assertEquals(0, client1.getNumOpenConnections()); @@ -271,7 +288,8 @@ public void testQueryInterruptionExceptionLogMessage() throws JsonProcessingExce EasyMock.expect( httpClient.go( EasyMock.capture(capturedRequest), - EasyMock.anyObject() + EasyMock.anyObject(), + EasyMock.anyObject(Duration.class) ) ) .andReturn(interruptionFuture) @@ -305,16 +323,15 @@ public void testQueryInterruptionExceptionLogMessage() throws JsonProcessingExce ); QueryableDruidServer queryableDruidServer = new QueryableDruidServer( - new DruidServer("test1", hostName, 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", hostName, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client1 ); serverSelector.addServerAndUpdateSegment(queryableDruidServer, dataSegment); TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); - HashMap context = Maps.newHashMap(); interruptionFuture.set(new ByteArrayInputStream("{\"error\":\"testing1\",\"errorMessage\":\"testing2\"}".getBytes())); - Sequence results = client1.run(query, context); + Sequence results = client1.run(query, defaultContext); QueryInterruptedException actualException = null; try { diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index 2a0a99a8125b..770e60444551 100644 --- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -30,7 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - import io.druid.client.BatchServerInventoryView; import io.druid.client.DruidServer; import io.druid.client.ServerView; @@ -39,8 +38,12 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.guava.Comparators; import io.druid.server.coordination.BatchDataSegmentAnnouncer; +import io.druid.server.coordination.CuratorDataSegmentServerAnnouncer; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; @@ -86,6 +89,7 @@ public class BatchServerInventoryViewTest private ObjectMapper jsonMapper; private Announcer announcer; private BatchDataSegmentAnnouncer segmentAnnouncer; + private DataSegmentServerAnnouncer serverAnnouncer; private Set testSegments; private BatchServerInventoryView batchServerInventoryView; private BatchServerInventoryView filteredBatchServerInventoryView; @@ -117,15 +121,34 @@ public void setUp() throws Exception ); announcer.start(); + DruidServerMetadata serverMetadata = new DruidServerMetadata( + "id", + "host", + Long.MAX_VALUE, + ServerType.HISTORICAL, + "tier", + 0 + ); + + ZkPathsConfig zkPathsConfig = new ZkPathsConfig() + { + @Override + public String getBase() + { + return testBasePath; + } + }; + + serverAnnouncer = new CuratorDataSegmentServerAnnouncer( + serverMetadata, + zkPathsConfig, + announcer, + jsonMapper + ); + serverAnnouncer.announce(); + segmentAnnouncer = new BatchDataSegmentAnnouncer( - new DruidServerMetadata( - "id", - "host", - Long.MAX_VALUE, - "type", - "tier", - 0 - ), + serverMetadata, new BatchDataSegmentAnnouncerConfig() { @Override @@ -134,18 +157,10 @@ public int getSegmentsPerNode() return 50; } }, - new ZkPathsConfig() - { - @Override - public String getBase() - { - return testBasePath; - } - }, + zkPathsConfig, announcer, jsonMapper ); - segmentAnnouncer.start(); testSegments = Sets.newConcurrentHashSet(); for (int i = 0; i < INITIAL_SEGMENTS; i++) { @@ -206,7 +221,7 @@ public void tearDown() throws Exception { batchServerInventoryView.stop(); filteredBatchServerInventoryView.stop(); - segmentAnnouncer.stop(); + serverAnnouncer.unannounce(); announcer.stop(); cf.close(); testingCluster.stop(); @@ -285,14 +300,8 @@ public void testRunWithFilterCallback() throws Exception Assert.assertEquals(testSegments, segments); ServerView.SegmentCallback callback = EasyMock.createStrictMock(ServerView.SegmentCallback.class); - Comparator dataSegmentComparator = new Comparator() - { - @Override - public int compare(DataSegment o1, DataSegment o2) - { - return o1.getInterval().equals(o2.getInterval()) ? 0 : -1; - } - }; + Comparator dataSegmentComparator = + Comparator.comparing(DataSegment::getInterval, Comparators.intervalsByStartThenEnd()); EasyMock .expect( @@ -435,7 +444,7 @@ public BatchDataSegmentAnnouncer call() "id", "host", Long.MAX_VALUE, - "type", + ServerType.HISTORICAL, "tier", 0 ), @@ -458,7 +467,6 @@ public String getBase() announcer, jsonMapper ); - segmentAnnouncer.start(); List segments = new ArrayList(); try { for (int j = 0; j < INITIAL_SEGMENTS / numThreads; ++j) { diff --git a/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java b/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java index 26f09e86b8cd..5c99abaa8882 100644 --- a/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java +++ b/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java @@ -24,6 +24,7 @@ import io.druid.client.ImmutableSegmentLoadInfo; import io.druid.jackson.DefaultObjectMapper; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import junit.framework.Assert; @@ -51,7 +52,7 @@ public void testSerde() throws IOException null, NoneShardSpec.instance(), 0, 0 - ), Sets.newHashSet(new DruidServerMetadata("a", "host", 10, "type", "tier", 1)) + ), Sets.newHashSet(new DruidServerMetadata("a", "host", 10, ServerType.HISTORICAL, "tier", 1)) ); ImmutableSegmentLoadInfo serde = mapper.readValue( diff --git a/server/src/test/java/io/druid/client/selector/ServerSelectorTest.java b/server/src/test/java/io/druid/client/selector/ServerSelectorTest.java index 0dc427906942..ffad1bd7facd 100644 --- a/server/src/test/java/io/druid/client/selector/ServerSelectorTest.java +++ b/server/src/test/java/io/druid/client/selector/ServerSelectorTest.java @@ -21,17 +21,29 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.druid.client.DirectDruidClient; +import io.druid.client.DruidServer; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; /** */ public class ServerSelectorTest { + TierSelectorStrategy tierSelectorStrategy; + + @Before + public void setUp() throws Exception + { + tierSelectorStrategy = EasyMock.createMock(TierSelectorStrategy.class); + EasyMock.expect(tierSelectorStrategy.getComparator()).andReturn(Integer::compare).anyTimes(); + } @Test public void testSegmentUpdate() throws Exception @@ -59,7 +71,10 @@ public void testSegmentUpdate() throws Exception ); selector.addServerAndUpdateSegment( - EasyMock.createMock(QueryableDruidServer.class), + new QueryableDruidServer( + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), + EasyMock.createMock(DirectDruidClient.class) + ), DataSegment.builder() .dataSource( "test_broker_server_view") diff --git a/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java b/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java index bc95553961c2..74757f0ff88a 100644 --- a/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java @@ -23,6 +23,7 @@ import io.druid.client.DirectDruidClient; import io.druid.client.DruidServer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -43,11 +44,11 @@ public void testHighestPriorityTierSelectorStrategy() { DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class); QueryableDruidServer lowPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client ); QueryableDruidServer highPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 1), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), client ); @@ -62,11 +63,11 @@ public void testLowestPriorityTierSelectorStrategy() { DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class); QueryableDruidServer lowPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client ); QueryableDruidServer highPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 1), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), client ); @@ -81,15 +82,15 @@ public void testCustomPriorityTierSelectorStrategy() { DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class); QueryableDruidServer lowPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, -1), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, -1), client ); QueryableDruidServer mediumPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0), client ); QueryableDruidServer highPriority = new QueryableDruidServer( - new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 1), + new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1), client ); diff --git a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 23db45847c35..7c3a9390a08c 100644 --- a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -30,6 +30,7 @@ import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; +import io.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -43,6 +44,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; public class IndexerSQLMetadataStorageCoordinatorTest { @@ -99,6 +101,66 @@ public class IndexerSQLMetadataStorageCoordinatorTest 100 ); + private final DataSegment numberedSegment0of0 = new DataSegment( + "fooDataSource", + Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(0, 0), + 9, + 100 + ); + + private final DataSegment numberedSegment1of0 = new DataSegment( + "fooDataSource", + Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(1, 0), + 9, + 100 + ); + + private final DataSegment numberedSegment2of0 = new DataSegment( + "fooDataSource", + Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(2, 0), + 9, + 100 + ); + + private final DataSegment numberedSegment2of1 = new DataSegment( + "fooDataSource", + Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(2, 1), + 9, + 100 + ); + + private final DataSegment numberedSegment3of1 = new DataSegment( + "fooDataSource", + Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), + "zversion", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(3, 1), + 9, + 100 + ); + private final Set SEGMENTS = ImmutableSet.of(defaultSegment, defaultSegment2); private final AtomicLong metadataUpdateCounter = new AtomicLong(); private IndexerSQLMetadataStorageCoordinator coordinator; @@ -713,4 +775,53 @@ public void testDeleteDataSourceMetadata() throws IOException Assert.assertNull("getDataSourceMetadataNullAfterDelete", coordinator.getDataSourceMetadata("fooDataSource")); } + + @Test + public void testSingleAdditionalNumberedShardWithNoCorePartitions() throws IOException + { + additionalNumberedShardTest(ImmutableSet.of(numberedSegment0of0)); + } + + @Test + public void testMultipleAdditionalNumberedShardsWithNoCorePartitions() throws IOException + { + additionalNumberedShardTest(ImmutableSet.of(numberedSegment0of0, numberedSegment1of0, numberedSegment2of0)); + } + + @Test + public void testSingleAdditionalNumberedShardWithOneCorePartition() throws IOException + { + additionalNumberedShardTest(ImmutableSet.of(numberedSegment2of1)); + } + + @Test + public void testMultipleAdditionalNumberedShardsWithOneCorePartition() throws IOException + { + additionalNumberedShardTest(ImmutableSet.of(numberedSegment2of1, numberedSegment3of1)); + } + + private void additionalNumberedShardTest(Set segments) throws IOException + { + coordinator.announceHistoricalSegments(segments); + + for (DataSegment segment : segments) { + Assert.assertArrayEquals( + mapper.writeValueAsString(segment).getBytes("UTF-8"), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + segment.getIdentifier() + ) + ); + } + + Assert.assertEquals( + segments.stream().map(DataSegment::getIdentifier).collect(Collectors.toList()), + getUsedIdentifiers() + ); + + // Should not update dataSource metadata. + Assert.assertEquals(0, metadataUpdateCounter.get()); + } } diff --git a/server/src/test/java/io/druid/query/LocatedSegmentDescriptorSerdeTest.java b/server/src/test/java/io/druid/query/LocatedSegmentDescriptorSerdeTest.java index e4c02119601c..aaa00c8701e1 100644 --- a/server/src/test/java/io/druid/query/LocatedSegmentDescriptorSerdeTest.java +++ b/server/src/test/java/io/druid/query/LocatedSegmentDescriptorSerdeTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -41,9 +42,9 @@ public void testDimensionsSpecSerde() throws Exception new SegmentDescriptor(new Interval(100, 200), "version", 100), 65535, Arrays.asList( - new DruidServerMetadata("server1", "host1", 30000L, "historical", "tier1", 0), - new DruidServerMetadata("server2", "host2", 40000L, "historical", "tier1", 1), - new DruidServerMetadata("server3", "host3", 50000L, "realtime", "tier2", 2) + new DruidServerMetadata("server1", "host1", 30000L, ServerType.HISTORICAL, "tier1", 0), + new DruidServerMetadata("server2", "host2", 40000L, ServerType.HISTORICAL, "tier1", 1), + new DruidServerMetadata("server3", "host3", 50000L, ServerType.REALTIME, "tier2", 2) ) ); diff --git a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java index 5e012a7e82f6..67dd673677e7 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java @@ -42,6 +42,7 @@ public class LookupIntrospectionResourceImplTest extends JerseyTest static LookupReferencesManager lookupReferencesManager = EasyMock.createMock(LookupReferencesManager.class); + @Override @Before public void setUp() throws Exception { diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java index 79d213186ccc..3904eca6bff4 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -33,6 +33,7 @@ import org.junit.Assert; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Iterator; @@ -52,7 +53,7 @@ public void testCombiningfirehose() throws IOException new ListFirehoseFactory(list2) ) ); - final Firehose firehose = combiningFactory.connect(null); + final Firehose firehose = combiningFactory.connect(null, null); for (int i = 1; i < 6; i++) { Assert.assertTrue(firehose.hasMore()); final InputRow inputRow = firehose.nextRow(); @@ -126,7 +127,7 @@ public static class ListFirehoseFactory implements FirehoseFactory iterator = rows.iterator(); return new Firehose() diff --git a/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java deleted file mode 100644 index f1631049c4cb..000000000000 --- a/server/src/test/java/io/druid/realtime/firehose/ReplayableFirehoseFactoryTest.java +++ /dev/null @@ -1,447 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.realtime.firehose; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.InputRow; -import io.druid.data.input.MapBasedInputRow; -import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.MapInputRowParser; -import io.druid.data.input.impl.TimeAndDimsParseSpec; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.parsers.ParseException; -import io.druid.segment.realtime.firehose.ReplayableFirehoseFactory; -import org.easymock.EasyMockSupport; -import org.easymock.IAnswer; -import org.joda.time.DateTime; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; - -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; - -public class ReplayableFirehoseFactoryTest extends EasyMockSupport -{ - private FirehoseFactory delegateFactory = createMock(FirehoseFactory.class); - private Firehose delegateFirehose = createMock(Firehose.class); - private InputRowParser parser = new MapInputRowParser(new TimeAndDimsParseSpec(null, null)); - private ObjectMapper mapper = new DefaultObjectMapper(); - - - private List testRows = Lists.newArrayList( - new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim1", "dim2"), - ImmutableMap.of("dim1", "val1", "dim2", "val2", "met1", 1) - ), - new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim1", "dim2"), - ImmutableMap.of("dim1", "val5", "dim2", "val2", "met1", 2) - ), - new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim2", "dim3"), - ImmutableMap.of("dim2", "val1", "dim3", "val2", "met1", 3) - ) - ); - - private ReplayableFirehoseFactory replayableFirehoseFactory; - - @Before - public void setup() - { - replayableFirehoseFactory = new ReplayableFirehoseFactory( - delegateFactory, - true, - 10000, - 3, - mapper - ); - } - - @Test - public void testConstructor() throws Exception - { - Assert.assertEquals(delegateFactory, replayableFirehoseFactory.getDelegateFactory()); - Assert.assertEquals(10000, replayableFirehoseFactory.getMaxTempFileSize()); - Assert.assertEquals(3, replayableFirehoseFactory.getReadFirehoseRetries()); - Assert.assertEquals(true, replayableFirehoseFactory.isReportParseExceptions()); - } - - @Test - public void testReplayableFirehoseNoEvents() throws Exception - { - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andReturn(false); - delegateFirehose.close(); - replayAll(); - - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - Assert.assertFalse(firehose.hasMore()); - } - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithEvents() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - - delegateFirehose.close(); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - // now replay! - rows.clear(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithoutReportParseExceptions() throws Exception - { - final boolean hasMore[] = {true}; - replayableFirehoseFactory = new ReplayableFirehoseFactory( - delegateFactory, - false, - 10000, - 3, - mapper - ); - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andThrow(new ParseException("unparseable!")) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - - delegateFirehose.close(); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test(expected = ParseException.class) - public void testReplayableFirehoseWithReportParseExceptions() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andThrow(new ParseException("unparseable!")) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - - delegateFirehose.close(); - replayAll(); - - replayableFirehoseFactory.connect(parser); - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithConnectRetries() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andThrow(new IOException()) - .andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - delegateFirehose.close(); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithNextRowRetries() throws Exception - { - final boolean hasMore[] = {true}; - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose).times(2); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)) - .andThrow(new RuntimeException()) - .andReturn(testRows.get(0)) - .andReturn(testRows.get(1)) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return testRows.get(2); - } - } - ); - delegateFirehose.close(); - expectLastCall().times(2); - replayAll(); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRows, rows); - - verifyAll(); - } - - @Test(expected = TestReadingException.class) - public void testReplayableFirehoseWithNoRetries() throws Exception - { - replayableFirehoseFactory = new ReplayableFirehoseFactory( - delegateFactory, - false, - 10000, - 0, - mapper - ); - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andReturn(true).times(2); - expect(delegateFirehose.nextRow()).andThrow(new TestReadingException()); - - delegateFirehose.close(); - expectLastCall(); - replayAll(); - - replayableFirehoseFactory.connect(parser); - verifyAll(); - } - - @Test - public void testReplayableFirehoseWithMultipleFiles() throws Exception - { - replayableFirehoseFactory = new ReplayableFirehoseFactory(delegateFactory, false, 1, 3, mapper); - - final boolean hasMore[] = {true}; - final int multiplicationFactor = 500; - - final InputRow finalRow = new MapBasedInputRow( - DateTime.now(), Lists.newArrayList("dim4", "dim5"), - ImmutableMap.of("dim4", "val12", "dim5", "val20", "met1", 30) - ); - - expect(delegateFactory.connect(parser)).andReturn(delegateFirehose); - expect(delegateFirehose.hasMore()).andAnswer( - new IAnswer() - { - @Override - public Boolean answer() throws Throwable - { - return hasMore[0]; - } - } - ).anyTimes(); - - expect(delegateFirehose.nextRow()) - .andReturn(testRows.get(0)).times(multiplicationFactor) - .andReturn(testRows.get(1)).times(multiplicationFactor) - .andReturn(testRows.get(2)).times(multiplicationFactor) - .andAnswer( - new IAnswer() - { - @Override - public InputRow answer() throws Throwable - { - hasMore[0] = false; - return finalRow; - } - } - ); - - delegateFirehose.close(); - replayAll(); - - List testRowsMultiplied = Lists.newArrayList(); - for (InputRow row : testRows) { - for (int i = 0; i < multiplicationFactor; i++) { - testRowsMultiplied.add(row); - } - } - testRowsMultiplied.add(finalRow); - - List rows = Lists.newArrayList(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRowsMultiplied, rows); - - // now replay! - rows.clear(); - try (Firehose firehose = replayableFirehoseFactory.connect(parser)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - Assert.assertEquals(testRowsMultiplied, rows); - - verifyAll(); - } - - private class TestReadingException extends RuntimeException - { - } -} - - - diff --git a/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java b/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java index a91e6538d68c..447bfddfe622 100644 --- a/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java +++ b/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java @@ -88,14 +88,14 @@ public void testPush() throws IOException Assert.assertEquals(dataSegment2, returnSegment2); Assert.assertNotEquals( - DataSegmentPusherUtil.getStorageDir(dataSegment), - DataSegmentPusherUtil.getStorageDir(dataSegment2) + localDataSegmentPusher.getStorageDir(dataSegment), + localDataSegmentPusher.getStorageDir(dataSegment2) ); for (DataSegment returnSegment : ImmutableList.of(returnSegment1, returnSegment2)) { File outDir = new File( config.getStorageDirectory(), - DataSegmentPusherUtil.getStorageDir(returnSegment) + localDataSegmentPusher.getStorageDir(returnSegment) ); File versionFile = new File(outDir, "index.zip"); File descriptorJson = new File(outDir, "descriptor.json"); diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 2484030ae135..d9e5823bc7b8 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -70,8 +70,10 @@ import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; import io.druid.segment.realtime.plumber.Sink; +import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.timeline.partition.LinearShardSpec; import io.druid.utils.Runnables; +import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -81,6 +83,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Iterator; @@ -143,7 +146,7 @@ public void setUp() throws Exception new FirehoseFactory() { @Override - public Firehose connect(InputRowParser parser) throws IOException + public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException { return new TestFirehose(rows.iterator()); } @@ -215,7 +218,8 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException tuningConfig ) ), - null + null, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class) ); plumber2 = new TestPlumber(new Sink( new Interval("0/P5000Y"), @@ -234,7 +238,8 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException tuningConfig ) ), - null + null, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class) ); tuningConfig_0 = new RealtimeTuningConfig( @@ -319,6 +324,7 @@ public void run() realtimeManager3 = new RealtimeManager( Arrays.asList(department_0, department_1), conglomerate, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), ImmutableMap.>of( "testing", ImmutableMap.of( diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java new file mode 100644 index 000000000000..1f87e0d6f85b --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java @@ -0,0 +1,464 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.AbstractFuture; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; +import io.druid.segment.incremental.IndexSizeExceededException; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.appenderator.AppenderatorDriverTest.TestCommitterSupplier; +import io.druid.segment.realtime.appenderator.AppenderatorDriverTest.TestSegmentAllocator; +import io.druid.segment.realtime.appenderator.AppenderatorDriverTest.TestSegmentHandoffNotifierFactory; +import io.druid.timeline.DataSegment; +import org.hamcrest.CoreMatchers; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +public class AppenderatorDriverFailTest +{ + private static final String DATA_SOURCE = "foo"; + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private static final long PUBLISH_TIMEOUT = 1000; + + private static final List ROWS = ImmutableList.of( + new MapBasedInputRow( + new DateTime("2000"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo", "met1", "1") + ), + new MapBasedInputRow( + new DateTime("2000T01"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo", "met1", 2.0) + ), + new MapBasedInputRow( + new DateTime("2000T01"), + ImmutableList.of("dim2"), + ImmutableMap.of("dim2", "bar", "met1", 2.0) + ) + ); + + SegmentAllocator allocator; + TestSegmentHandoffNotifierFactory segmentHandoffNotifierFactory; + AppenderatorDriver driver; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Before + public void setUp() + { + allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); + segmentHandoffNotifierFactory = new TestSegmentHandoffNotifierFactory(); + } + + @After + public void tearDown() throws Exception + { + if (driver != null) { + driver.clear(); + driver.close(); + } + } + + @Test + public void testFailDuringPersist() throws IOException, InterruptedException, TimeoutException, ExecutionException + { + expectedException.expect(TimeoutException.class); + + driver = new AppenderatorDriver( + createPersistFailAppenderator(), + allocator, + segmentHandoffNotifierFactory, + new NoopUsedSegmentChecker(), + OBJECT_MAPPER, + new FireDepartmentMetrics() + ); + + driver.startJob(); + + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + segmentHandoffNotifierFactory.setHandoffDelay(100); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); + } + + driver.publish( + AppenderatorDriverTest.makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + } + + @Test + public void testInterruptDuringPush() throws IOException, InterruptedException, TimeoutException, ExecutionException + { + expectedException.expect(ExecutionException.class); + expectedException.expectCause(CoreMatchers.instanceOf(InterruptedException.class)); + + driver = new AppenderatorDriver( + createPushInterruptAppenderator(), + allocator, + segmentHandoffNotifierFactory, + new NoopUsedSegmentChecker(), + OBJECT_MAPPER, + new FireDepartmentMetrics() + ); + + driver.startJob(); + + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + segmentHandoffNotifierFactory.setHandoffDelay(100); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); + } + + driver.publish( + AppenderatorDriverTest.makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + } + + @Test + public void testFailDuringPush() throws IOException, InterruptedException, TimeoutException, ExecutionException + { + expectedException.expect(TimeoutException.class); + + driver = new AppenderatorDriver( + createPushFailAppenderator(), + allocator, + segmentHandoffNotifierFactory, + new NoopUsedSegmentChecker(), + OBJECT_MAPPER, + new FireDepartmentMetrics() + ); + + driver.startJob(); + + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + segmentHandoffNotifierFactory.setHandoffDelay(100); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); + } + + driver.publish( + AppenderatorDriverTest.makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + } + + @Test + public void testFailDuringDrop() throws IOException, InterruptedException, TimeoutException, ExecutionException + { + expectedException.expect(ExecutionException.class); + expectedException.expectCause(CoreMatchers.instanceOf(ISE.class)); + expectedException.expectMessage( + "Fail test while dropping segment[foo_2000-01-01T00:00:00.000Z_2000-01-01T01:00:00.000Z_abc123]" + ); + + driver = new AppenderatorDriver( + createDropFailAppenderator(), + allocator, + segmentHandoffNotifierFactory, + new NoopUsedSegmentChecker(), + OBJECT_MAPPER, + new FireDepartmentMetrics() + ); + + driver.startJob(); + + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + segmentHandoffNotifierFactory.setHandoffDelay(100); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); + } + + final SegmentsAndMetadata published = driver.publish( + AppenderatorDriverTest.makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + + driver.registerHandoff(published).get(); + } + + private static class NoopUsedSegmentChecker implements UsedSegmentChecker + { + @Override + public Set findUsedSegments(Set identifiers) throws IOException + { + return ImmutableSet.of(); + } + } + + static Appenderator createPushFailAppenderator() + { + return new FailableAppenderator().disablePush(); + } + + static Appenderator createPushInterruptAppenderator() + { + return new FailableAppenderator().interruptPush(); + } + + static Appenderator createPersistFailAppenderator() + { + return new FailableAppenderator().disablePersist(); + } + + static Appenderator createDropFailAppenderator() + { + return new FailableAppenderator().disableDrop(); + } + + private static class FailableAppenderator implements Appenderator + { + private final Map> rows = new HashMap<>(); + + private boolean dropEnabled = true; + private boolean persistEnabled = true; + private boolean pushEnabled = true; + private boolean interruptPush = false; + + private int numRows; + + public FailableAppenderator disableDrop() + { + dropEnabled = false; + return this; + } + + public FailableAppenderator disablePersist() + { + persistEnabled = false; + return this; + } + + public FailableAppenderator disablePush() + { + pushEnabled = false; + interruptPush = false; + return this; + } + + public FailableAppenderator interruptPush() + { + pushEnabled = false; + interruptPush = true; + return this; + } + + @Override + public String getDataSource() + { + return null; + } + + @Override + public Object startJob() + { + return null; + } + + @Override + public int add( + SegmentIdentifier identifier, InputRow row, Supplier committerSupplier + ) throws IndexSizeExceededException, SegmentNotWritableException + { + rows.computeIfAbsent(identifier, k -> new ArrayList<>()).add(row); + return ++numRows; + } + + @Override + public List getSegments() + { + return ImmutableList.copyOf(rows.keySet()); + } + + @Override + public int getRowCount(SegmentIdentifier identifier) + { + final List rows = this.rows.get(identifier); + if (rows != null) { + return rows.size(); + } else { + return 0; + } + } + + @Override + public int getTotalRowCount() + { + return numRows; + } + + @Override + public void clear() throws InterruptedException + { + rows.clear(); + } + + @Override + public ListenableFuture drop(SegmentIdentifier identifier) + { + if (dropEnabled) { + rows.remove(identifier); + return Futures.immediateFuture(null); + } else { + return Futures.immediateFailedFuture(new ISE("Fail test while dropping segment[%s]", identifier)); + } + } + + @Override + public ListenableFuture persist( + Collection identifiers, Committer committer + ) + { + if (persistEnabled) { + // do nothing + return Futures.immediateFuture(committer.getMetadata()); + } else { + return Futures.immediateFailedFuture(new ISE("Fail test while persisting segments[%s]", identifiers)); + } + } + + @Override + public ListenableFuture push( + Collection identifiers, Committer committer + ) + { + if (pushEnabled) { + final List segments = identifiers.stream() + .map( + id -> new DataSegment( + id.getDataSource(), + id.getInterval(), + id.getVersion(), + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of(), + id.getShardSpec(), + 0, + 0 + ) + ) + .collect(Collectors.toList()); + return Futures.transform( + persist(identifiers, committer), + (Function) commitMetadata -> new SegmentsAndMetadata(segments, commitMetadata) + ); + } else { + if (interruptPush) { + return new AbstractFuture() + { + @Override + public SegmentsAndMetadata get(long timeout, TimeUnit unit) + throws InterruptedException, TimeoutException, ExecutionException + { + throw new InterruptedException("Interrupt test while pushing segments"); + } + + @Override + public SegmentsAndMetadata get() throws InterruptedException, ExecutionException + { + throw new InterruptedException("Interrupt test while pushing segments"); + } + }; + } else { + return Futures.immediateFailedFuture(new ISE("Fail test while pushing segments[%s]", identifiers)); + } + } + } + + @Override + public void close() + { + + } + + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, Iterable intervals + ) + { + throw new UnsupportedOperationException(); + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, Iterable specs + ) + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java similarity index 51% rename from server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java rename to server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java index 26ad0fa052b6..ff284ac0dd3a 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverTest.java @@ -29,11 +29,11 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.query.SegmentDescriptor; @@ -57,19 +57,22 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -public class FiniteAppenderatorDriverTest +public class AppenderatorDriverTest { private static final String DATA_SOURCE = "foo"; private static final String VERSION = "abc123"; private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - private static final int MAX_ROWS_IN_MEMORY = 4; + private static final int MAX_ROWS_IN_MEMORY = 10; private static final int MAX_ROWS_PER_SEGMENT = 3; - private static final long MAX_PERSISTED_SEGMENTS_BYTES = 1024; - private static final long HANDOFF_CONDITION_TIMEOUT = 0; + private static final long PUBLISH_TIMEOUT = 1000; + private static final long HANDOFF_CONDITION_TIMEOUT = 1000; private static final List ROWS = Arrays.asList( new MapBasedInputRow( @@ -89,24 +92,25 @@ public class FiniteAppenderatorDriverTest ) ); - SegmentAllocator allocator; - AppenderatorTester appenderatorTester; - FiniteAppenderatorDriver driver; + private SegmentAllocator allocator; + private AppenderatorTester appenderatorTester; + private Appenderator appenderator; + private TestSegmentHandoffNotifierFactory segmentHandoffNotifierFactory; + private AppenderatorDriver driver; @Before public void setUp() { appenderatorTester = new AppenderatorTester(MAX_ROWS_IN_MEMORY); + appenderator = appenderatorTester.getAppenderator(); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); - driver = new FiniteAppenderatorDriver( - appenderatorTester.getAppenderator(), + segmentHandoffNotifierFactory = new TestSegmentHandoffNotifierFactory(); + driver = new AppenderatorDriver( + appenderator, allocator, - new TestSegmentHandoffNotifierFactory(), + segmentHandoffNotifierFactory, new TestUsedSegmentChecker(), OBJECT_MAPPER, - MAX_ROWS_PER_SEGMENT, - MAX_PERSISTED_SEGMENTS_BYTES, - HANDOFF_CONDITION_TIMEOUT, new FireDepartmentMetrics() ); } @@ -127,13 +131,16 @@ public void testSimple() throws Exception for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); - Assert.assertNotNull(driver.add(ROWS.get(i), "dummy", committerSupplier, makeOkPublisher(), false)); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); } - final SegmentsAndMetadata segmentsAndMetadata = driver.publishAndWaitHandoff( + final SegmentsAndMetadata published = driver.publish( makeOkPublisher(), - committerSupplier.get() - ); + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) + .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals( ImmutableSet.of( @@ -158,75 +165,200 @@ public void testMaxRowsPerSegment() throws Exception InputRow row = new MapBasedInputRow( new DateTime("2000T01"), ImmutableList.of("dim2"), - ImmutableMap.of( + ImmutableMap.of( "dim2", String.format("bar-%d", i), "met1", 2.0 ) ); - Assert.assertNotNull(driver.add(row, "dummy", committerSupplier, makeOkPublisher(), false)); + final AppenderatorDriverAddResult addResult = driver.add(row, "dummy", committerSupplier); + Assert.assertTrue(addResult.isOk()); + if (addResult.getNumRowsInSegment() > MAX_ROWS_PER_SEGMENT) { + driver.moveSegmentOut("dummy", ImmutableList.of(addResult.getSegmentIdentifier())); + } } - final SegmentsAndMetadata segmentsAndMetadata = driver.publishAndWaitHandoff( + final SegmentsAndMetadata published = driver.publish( makeOkPublisher(), - committerSupplier.get() - ); + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) + .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertEquals(numSegments, segmentsAndMetadata.getSegments().size()); Assert.assertEquals(numSegments * MAX_ROWS_PER_SEGMENT, segmentsAndMetadata.getCommitMetadata()); } + @Test(timeout = 5000L, expected = TimeoutException.class) + public void testHandoffTimeout() throws Exception + { + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + segmentHandoffNotifierFactory.disableHandoff(); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); + } + + final SegmentsAndMetadata published = driver.publish( + makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + driver.registerHandoff(published).get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); + } + @Test - public void testMaxPersistedSegmentsBytes() throws Exception + public void testPublishPerRow() throws IOException, InterruptedException, TimeoutException, ExecutionException { final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + Assert.assertNull(driver.startJob()); - for (int i = 0; i < 7; i++) { // i = 7 + // Add the first row and publish immediately + { + committerSupplier.setMetadata(1); + Assert.assertTrue(driver.add(ROWS.get(0), "dummy", committerSupplier).isOk()); + + final SegmentsAndMetadata published = driver.publish( + makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) + .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); + Assert.assertEquals( + ImmutableSet.of( + new SegmentIdentifier(DATA_SOURCE, new Interval("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)) + ), + asIdentifiers(segmentsAndMetadata.getSegments()) + ); + + Assert.assertEquals(1, segmentsAndMetadata.getCommitMetadata()); + } + + // Add the second and third rows and publish immediately + for (int i = 1; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); - InputRow row = new MapBasedInputRow( - new DateTime("2000T01"), - ImmutableList.of("dim2"), - ImmutableMap.of( - "dim2", - String.format("bar-%d", i), - "met1", - 2.0 - ) + Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); + + final SegmentsAndMetadata published = driver.publish( + makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) + .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); + Assert.assertEquals( + ImmutableSet.of( + // The second and third rows have the same dataSource, interval, and version, but different shardSpec of + // different partitionNum + new SegmentIdentifier(DATA_SOURCE, new Interval("2000T01/PT1H"), VERSION, new NumberedShardSpec(i - 1, 0)) + ), + asIdentifiers(segmentsAndMetadata.getSegments()) ); - Assert.assertNotNull(driver.add(row, "dummy", committerSupplier, makeOkPublisher(), true)); + + Assert.assertEquals(i + 1, segmentsAndMetadata.getCommitMetadata()); } - committerSupplier.setMetadata(8); - InputRow row = new MapBasedInputRow( - new DateTime("2000T01"), - ImmutableList.of("dim2"), - ImmutableMap.of( - "dim2", - String.format("bar-%d", 7), - "met1", - 2.0 - ) - ); - final Pair> pair = driver.add( - row, - "dummy", - committerSupplier, + driver.persist(committerSupplier.get()); + + // There is no remaining rows in the driver, and thus the result must be empty + final SegmentsAndMetadata published = driver.publish( makeOkPublisher(), - true + committerSupplier.get(), + ImmutableList.of("dummy") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + final SegmentsAndMetadata segmentsAndMetadata = driver.registerHandoff(published) + .get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + ImmutableSet.of(), + asIdentifiers(segmentsAndMetadata.getSegments()) ); - final SegmentIdentifier added = pair.lhs; - final List movedOut = pair.rhs; - Assert.assertNotNull(added); - Assert.assertEquals(3, movedOut.size()); + Assert.assertEquals(3, segmentsAndMetadata.getCommitMetadata()); + } + + @Test + public void testIncrementalHandoff() throws Exception + { + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + + Assert.assertNull(driver.startJob()); + + committerSupplier.setMetadata(1); + Assert.assertTrue(driver.add(ROWS.get(0), "sequence_0", committerSupplier).isOk()); + + for (int i = 1; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "sequence_1", committerSupplier).isOk()); + } - final SegmentsAndMetadata segmentsAndMetadata = driver.publishAndWaitHandoff( + final ListenableFuture futureForSequence0 = driver.publishAndRegisterHandoff( makeOkPublisher(), - committerSupplier.get() + committerSupplier.get(), + ImmutableList.of("sequence_0") ); - Assert.assertEquals(0, segmentsAndMetadata.getSegments().size()); - Assert.assertEquals(8, segmentsAndMetadata.getCommitMetadata()); + + final ListenableFuture futureForSequence1 = driver.publishAndRegisterHandoff( + makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("sequence_1") + ); + + final SegmentsAndMetadata handedoffFromSequence0 = futureForSequence0.get( + HANDOFF_CONDITION_TIMEOUT, + TimeUnit.MILLISECONDS + ); + final SegmentsAndMetadata handedoffFromSequence1 = futureForSequence1.get( + HANDOFF_CONDITION_TIMEOUT, + TimeUnit.MILLISECONDS + ); + + Assert.assertEquals( + ImmutableSet.of( + new SegmentIdentifier(DATA_SOURCE, new Interval("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)) + ), + asIdentifiers(handedoffFromSequence0.getSegments()) + ); + + Assert.assertEquals( + ImmutableSet.of( + new SegmentIdentifier(DATA_SOURCE, new Interval("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)) + ), + asIdentifiers(handedoffFromSequence1.getSegments()) + ); + + Assert.assertEquals(3, handedoffFromSequence0.getCommitMetadata()); + Assert.assertEquals(3, handedoffFromSequence1.getCommitMetadata()); + } + + @Test + public void testRegisterHandoff() throws IOException, InterruptedException, ExecutionException, TimeoutException + { + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertTrue(driver.add(ROWS.get(i), "dummy" + i, committerSupplier).isOk()); + } + + driver.persist(committerSupplier.get()); + + final SegmentsAndMetadata published = driver.publish( + makeOkPublisher(), + committerSupplier.get(), + ImmutableList.of("dummy0") + ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); + driver.registerHandoff(published).get(HANDOFF_CONDITION_TIMEOUT, TimeUnit.MILLISECONDS); + + Assert.assertEquals(2, appenderator.getTotalRowCount()); } private Set asIdentifiers(Iterable segments) @@ -246,7 +378,7 @@ public SegmentIdentifier apply(DataSegment input) ); } - private TransactionalSegmentPublisher makeOkPublisher() + static TransactionalSegmentPublisher makeOkPublisher() { return new TransactionalSegmentPublisher() { @@ -258,7 +390,7 @@ public boolean publishSegments(Set segments, Object commitMetadata) }; } - private class TestCommitterSupplier implements Supplier + static class TestCommitterSupplier implements Supplier { private final AtomicReference metadata = new AtomicReference<>(); @@ -288,7 +420,7 @@ public void run() } } - private class TestSegmentAllocator implements SegmentAllocator + static class TestSegmentAllocator implements SegmentAllocator { private final String dataSource; private final Granularity granularity; @@ -302,14 +434,13 @@ public TestSegmentAllocator(String dataSource, Granularity granularity) @Override public SegmentIdentifier allocate( - final DateTime timestamp, final InputRow row, final String sequenceName, final String previousSegmentId ) throws IOException { synchronized (counters) { - final long timestampTruncated = granularity.bucketStart(timestamp).getMillis(); + final long timestampTruncated = granularity.bucketStart(row.getTimestamp()).getMillis(); if (!counters.containsKey(timestampTruncated)) { counters.put(timestampTruncated, new AtomicInteger()); } @@ -324,8 +455,21 @@ public SegmentIdentifier allocate( } } - private class TestSegmentHandoffNotifierFactory implements SegmentHandoffNotifierFactory + static class TestSegmentHandoffNotifierFactory implements SegmentHandoffNotifierFactory { + private boolean handoffEnabled = true; + private long handoffDelay; + + public void disableHandoff() + { + handoffEnabled = false; + } + + public void setHandoffDelay(long delay) + { + handoffDelay = delay; + } + @Override public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) { @@ -338,8 +482,19 @@ public boolean registerSegmentHandoffCallback( final Runnable handOffRunnable ) { - // Immediate handoff - exec.execute(handOffRunnable); + if (handoffEnabled) { + + if (handoffDelay > 0) { + try { + Thread.sleep(handoffDelay); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + exec.execute(handOffRunnable); + } return true; } diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 2a4490056a04..a6baa240fe07 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -180,6 +180,8 @@ public void run() Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 1), committerSupplier); Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); + appenderator.persist(ImmutableList.of(IDENTIFIERS.get(1)), committerSupplier.get()); + Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); appenderator.close(); Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); } @@ -241,44 +243,44 @@ public void run() } @Test(timeout = 10000L) - public void testPersistedBytes() throws Exception + public void testTotalRowCount() throws Exception { try (final AppenderatorTester tester = new AppenderatorTester(3)) { final Appenderator appenderator = tester.getAppenderator(); final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(0, appenderator.getTotalRowCount()); appenderator.startJob(); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(0, appenderator.getTotalRowCount()); appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(1, appenderator.getTotalRowCount()); appenderator.add(IDENTIFIERS.get(1), IR("2000", "bar", 1), committerSupplier); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(2, appenderator.getTotalRowCount()); appenderator.persistAll(committerSupplier.get()).get(); - Assert.assertEquals(4456, appenderator.getPersistedBytes()); + Assert.assertEquals(2, appenderator.getTotalRowCount()); appenderator.drop(IDENTIFIERS.get(0)).get(); - Assert.assertEquals(2228, appenderator.getPersistedBytes()); + Assert.assertEquals(1, appenderator.getTotalRowCount()); appenderator.drop(IDENTIFIERS.get(1)).get(); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(0, appenderator.getTotalRowCount()); appenderator.add(IDENTIFIERS.get(2), IR("2001", "bar", 1), committerSupplier); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(1, appenderator.getTotalRowCount()); appenderator.add(IDENTIFIERS.get(2), IR("2001", "baz", 1), committerSupplier); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(2, appenderator.getTotalRowCount()); appenderator.add(IDENTIFIERS.get(2), IR("2001", "qux", 1), committerSupplier); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(3, appenderator.getTotalRowCount()); appenderator.add(IDENTIFIERS.get(2), IR("2001", "bob", 1), committerSupplier); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(4, appenderator.getTotalRowCount()); appenderator.persistAll(committerSupplier.get()).get(); - Assert.assertEquals(4522, appenderator.getPersistedBytes()); + Assert.assertEquals(4, appenderator.getTotalRowCount()); appenderator.drop(IDENTIFIERS.get(2)).get(); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(0, appenderator.getTotalRowCount()); appenderator.close(); - Assert.assertEquals(0, appenderator.getPersistedBytes()); + Assert.assertEquals(0, appenderator.getTotalRowCount()); } } diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index fba52b2eacf8..98399784c511 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -61,6 +61,7 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; @@ -186,6 +187,12 @@ public DataSegment push(File file, DataSegment segment) throws IOException pushedSegments.add(segment); return segment; } + + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } }; appenderator = Appenderators.createRealtime( schema, @@ -235,12 +242,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio { } - - @Override - public boolean isAnnounced(DataSegment segment) - { - return false; - } }, emitter, queryExecutor, diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index 3517e7eb3e18..2e9a7d3716d6 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - import io.druid.concurrent.Execs; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; @@ -88,7 +87,8 @@ public void setUp() throws Exception null, null ) - ) + ), + null ); } @@ -220,7 +220,8 @@ public void testDuplicateRegistering() throws IOException null, null ) - ) + ), + null ); } diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index ea8b5590701a..01a90fa8b8bc 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -22,13 +22,20 @@ import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import io.druid.data.input.impl.CSVParseSpec; +import io.druid.collections.spatial.search.RadiusBound; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DelimitedParseSpec; import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.NewSpatialDimensionSchema; +import io.druid.data.input.impl.StringDimensionSchema; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; +import io.druid.hll.HyperLogLogCollector; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.filter.SpatialDimFilter; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -37,7 +44,11 @@ import io.druid.segment.StorageAdapter; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import io.druid.segment.incremental.OnheapIncrementalIndex; +import org.joda.time.DateTime; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -50,6 +61,33 @@ */ public class IngestSegmentFirehoseTest { + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("host"), + new NewSpatialDimensionSchema("spatial", ImmutableList.of("x", "y")) + ), + null, + null + ); + + private static final DimensionsSpec DIMENSIONS_SPEC_REINDEX = new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("host"), + new NewSpatialDimensionSchema("spatial", ImmutableList.of("spatial")) + ), + null, + null + ); + + private static final List AGGREGATORS = ImmutableList.of( + new LongSumAggregatorFactory("visited_sum", "visited"), + new HyperUniquesAggregatorFactory("unique_hosts", "host") + ); + + private static final List AGGREGATORS_REINDEX = ImmutableList.of( + new LongSumAggregatorFactory("visited_sum", "visited_sum"), + new HyperUniquesAggregatorFactory("unique_hosts", "unique_hosts") + ); @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -58,78 +96,117 @@ public class IngestSegmentFirehoseTest private IndexMerger indexMerger = TestHelper.getTestIndexMerger(); @Test - public void testSanity() throws Exception + public void testReadFromIndexAndWriteAnotherIndex() throws Exception { + // Tests a "reindexing" use case that is a common use of ingestSegment. + File segmentDir = tempFolder.newFolder(); createTestIndex(segmentDir); - QueryableIndex qi = null; - try { - qi = indexIO.loadIndex(segmentDir); - StorageAdapter sa = new QueryableIndexStorageAdapter(qi); - WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); - IngestSegmentFirehose firehose = new IngestSegmentFirehose( + try ( + final QueryableIndex qi = indexIO.loadIndex(segmentDir); + final IncrementalIndex index = new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec(DIMENSIONS_SPEC_REINDEX) + .withQueryGranularity(Granularities.NONE) + .withMetrics(AGGREGATORS_REINDEX.toArray(new AggregatorFactory[]{})) + .build(), + true, + 5000 + ) + ) { + final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); + final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); + final IngestSegmentFirehose firehose = new IngestSegmentFirehose( ImmutableList.of(wsa, wsa), - ImmutableList.of("host"), + ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), null ); int count = 0; while (firehose.hasMore()) { - firehose.nextRow(); + final InputRow row = firehose.nextRow(); + Assert.assertNotNull(row); + if (count == 0) { + Assert.assertEquals(new DateTime("2014-10-22T00Z"), row.getTimestamp()); + Assert.assertEquals("host1", row.getRaw("host")); + Assert.assertEquals("0,1", row.getRaw("spatial")); + Assert.assertEquals(10L, row.getRaw("visited_sum")); + Assert.assertEquals(1.0d, ((HyperLogLogCollector) row.getRaw("unique_hosts")).estimateCardinality(), 0.1); + } count++; + index.add(row); } Assert.assertEquals(18, count); - } - finally { - if (qi != null) { - qi.close(); - } + + // Check the index + Assert.assertEquals(9, index.size()); + final IncrementalIndexStorageAdapter queryable = new IncrementalIndexStorageAdapter(index); + Assert.assertEquals(2, queryable.getAvailableDimensions().size()); + Assert.assertEquals("host", queryable.getAvailableDimensions().get(0)); + Assert.assertEquals("spatial", queryable.getAvailableDimensions().get(1)); + Assert.assertEquals(ImmutableList.of("visited_sum", "unique_hosts"), queryable.getAvailableMetrics()); + + // Do a spatial filter + final IngestSegmentFirehose firehose2 = new IngestSegmentFirehose( + ImmutableList.of(new WindowedStorageAdapter(queryable, new Interval("2000/3000"))), + ImmutableList.of("host", "spatial"), + ImmutableList.of("visited_sum", "unique_hosts"), + new SpatialDimFilter("spatial", new RadiusBound(new float[]{1, 0}, 0.1f)) + ); + final InputRow row = firehose2.nextRow(); + Assert.assertFalse(firehose2.hasMore()); + Assert.assertEquals(new DateTime("2014-10-22T00Z"), row.getTimestamp()); + Assert.assertEquals("host2", row.getRaw("host")); + Assert.assertEquals("1,0", row.getRaw("spatial")); + Assert.assertEquals(40L, row.getRaw("visited_sum")); + Assert.assertEquals(1.0d, ((HyperLogLogCollector) row.getRaw("unique_hosts")).estimateCardinality(), 0.1); } } private void createTestIndex(File segmentDir) throws Exception { - List rows = Lists.newArrayList( - "2014102200,host1,10", - "2014102200,host2,20", - "2014102200,host3,30", - "2014102201,host1,10", - "2014102201,host2,20", - "2014102201,host3,30", - "2014102202,host1,10", - "2014102202,host2,20", - "2014102202,host3,30" + final List rows = Lists.newArrayList( + "2014102200\thost1\t10\t0\t1", + "2014102200\thost2\t20\t1\t0", + "2014102200\thost3\t30\t1\t1", + "2014102201\thost1\t10\t1\t1", + "2014102201\thost2\t20\t1\t1", + "2014102201\thost3\t30\t1\t1", + "2014102202\thost1\t10\t1\t1", + "2014102202\thost2\t20\t1\t1", + "2014102202\thost3\t30\t1\t1" ); - StringInputRowParser parser = new StringInputRowParser( - new CSVParseSpec( + final StringInputRowParser parser = new StringInputRowParser( + new DelimitedParseSpec( new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), + DIMENSIONS_SPEC, + "\t", null, - ImmutableList.of("timestamp", "host", "visited") + ImmutableList.of("timestamp", "host", "visited", "x", "y", "spatial"), + false, + 0 ), Charsets.UTF_8.toString() ); - AggregatorFactory[] aggregators = new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited") - }; - - IncrementalIndex index = null; - try { - index = new OnheapIncrementalIndex(0, Granularities.NONE, aggregators, true, true, true, 5000); + try ( + final IncrementalIndex index = new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec(parser.getParseSpec().getDimensionsSpec()) + .withQueryGranularity(Granularities.NONE) + .withMetrics(AGGREGATORS.toArray(new AggregatorFactory[]{})) + .build(), + true, + 5000 + ) + ) { for (String line : rows) { index.add(parser.parse(line)); } indexMerger.persist(index, segmentDir, new IndexSpec()); } - finally { - if (index != null) { - index.close(); - } - } } - } diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java new file mode 100644 index 000000000000..99d711fdecb3 --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.firehose; + +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import io.druid.data.input.Firehose; +import io.druid.data.input.Row; +import io.druid.data.input.impl.CSVParseSpec; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.BufferedWriter; +import java.io.FileWriter; +import java.io.IOException; +import java.io.Writer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +public class LocalFirehoseFactoryTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private LocalFirehoseFactory factory; + + @Before + public void setup() throws IOException + { + for (int i = 0; i < 5; i++) { + try (final Writer writer = new BufferedWriter( + new FileWriter(temporaryFolder.newFile("test_" + i)) + )) { + writer.write((20171225 + i) + "," + i + "th test file\n"); + } + } + + for (int i = 0; i < 5; i++) { + try (final Writer writer = new BufferedWriter( + new FileWriter(temporaryFolder.newFile("filtered_" + i)) + )) { + writer.write((20171225 + i) + "," + i + "th filtered file\n"); + } + } + + factory = new LocalFirehoseFactory(temporaryFolder.getRoot(), "test_*", null); + } + + @Test + public void testConnect() throws IOException + { + try (final Firehose firehose = factory.connect(new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec( + "timestamp", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a")), + Lists.newArrayList(), + Lists.newArrayList() + ), + ",", + Arrays.asList("timestamp", "a"), + false, + 0 + ), + Charsets.UTF_8.name() + ), null)) { + final List rows = new ArrayList<>(); + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + + Assert.assertEquals(5, rows.size()); + rows.sort(Comparator.comparing(Row::getTimestamp)); + for (int i = 0; i < 5; i++) { + final List dimVals = rows.get(i).getDimension("a"); + Assert.assertEquals(1, dimVals.size()); + Assert.assertEquals(i + "th test file", dimVals.get(0)); + } + } + } +} diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java index 62c2d4defb68..bd9941f5b013 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java @@ -36,7 +36,7 @@ @RunWith(EasyMockRunner.class) public class ServiceAnnouncingChatHandlerProviderTest extends EasyMockSupport { - private class TestChatHandler implements ChatHandler {} + private static class TestChatHandler implements ChatHandler {} private static final String TEST_SERVICE_NAME = "test-service-name"; private static final String TEST_HOST = "test-host"; diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java index 5b67b9ee2ba2..8238366d6093 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -26,6 +26,7 @@ import io.druid.client.coordinator.CoordinatorClient; import io.druid.query.SegmentDescriptor; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import junit.framework.Assert; @@ -318,15 +319,15 @@ public void testHandoffChecksForInterval() private DruidServerMetadata createRealtimeServerMetadata(String name) { - return createServerMetadata(name, "realtime"); + return createServerMetadata(name, ServerType.REALTIME); } private DruidServerMetadata createHistoricalServerMetadata(String name) { - return createServerMetadata(name, "historical"); + return createServerMetadata(name, ServerType.HISTORICAL); } - private DruidServerMetadata createServerMetadata(String name, String type) + private DruidServerMetadata createServerMetadata(String name, ServerType type) { return new DruidServerMetadata( name, diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index 0ac353aba182..6298ac108409 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -78,6 +78,7 @@ public class AsyncQueryForwardingServletTest extends BaseJettyTest private static int port1; private static int port2; + @Override @Before public void setup() throws Exception { diff --git a/server/src/test/java/io/druid/server/ClientInfoResourceTest.java b/server/src/test/java/io/druid/server/ClientInfoResourceTest.java index af02fad8e669..6818f625c12e 100644 --- a/server/src/test/java/io/druid/server/ClientInfoResourceTest.java +++ b/server/src/test/java/io/druid/server/ClientInfoResourceTest.java @@ -43,9 +43,12 @@ import io.druid.client.DruidServer; import io.druid.client.FilteredServerInventoryView; import io.druid.client.TimelineServerView; +import io.druid.client.selector.HighestPriorityTierSelectorStrategy; +import io.druid.client.selector.RandomServerSelectorStrategy; import io.druid.client.selector.ServerSelector; import io.druid.query.TableDataSource; import io.druid.query.metadata.SegmentMetadataQueryConfig; +import io.druid.server.coordination.ServerType; import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; @@ -79,7 +82,7 @@ public class ClientInfoResourceTest public void setup() { VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - DruidServer server = new DruidServer("name", "host", 1234, "type", "tier", 0); + DruidServer server = new DruidServer("name", "host", 1234, ServerType.HISTORICAL, "tier", 0); addSegment(timeline, server, "1960-02-13/1961-02-14", ImmutableList.of("d5"), ImmutableList.of("m5"), "v0"); @@ -377,7 +380,7 @@ private void addSegment( .size(1) .build(); server.addDataSegment(segment.getIdentifier(), segment); - ServerSelector ss = new ServerSelector(segment, null); + ServerSelector ss = new ServerSelector(segment, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())); timeline.add(new Interval(interval), version, new SingleElementPartitionChunk(ss)); } @@ -401,7 +404,7 @@ private void addSegmentWithShardSpec( .size(1) .build(); server.addDataSegment(segment.getIdentifier(), segment); - ServerSelector ss = new ServerSelector(segment, null); + ServerSelector ss = new ServerSelector(segment, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())); timeline.add(new Interval(interval), version, shardSpec.createChunk(ss)); } diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index f8ca466b97ac..3a4573f7cc97 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -32,6 +32,7 @@ import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; @@ -95,11 +96,9 @@ public QueryRunner getQueryRunnerForIntervals( return new QueryRunner() { @Override - public Sequence run( - Query query, Map responseContext - ) + public Sequence run(QueryPlus query, Map responseContext) { - return Sequences.empty(); + return Sequences.empty(); } }; } diff --git a/server/src/test/java/io/druid/server/SegmentManagerTest.java b/server/src/test/java/io/druid/server/SegmentManagerTest.java new file mode 100644 index 000000000000..3176b00368a1 --- /dev/null +++ b/server/src/test/java/io/druid/server/SegmentManagerTest.java @@ -0,0 +1,439 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import io.druid.java.util.common.MapUtils; +import io.druid.segment.AbstractSegment; +import io.druid.segment.QueryableIndex; +import io.druid.segment.ReferenceCountingSegment; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; +import io.druid.segment.loading.SegmentLoader; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.server.SegmentManager.DataSourceState; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NoneShardSpec; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +public class SegmentManagerTest +{ + private static final SegmentLoader segmentLoader = new SegmentLoader() + { + @Override + public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException + { + return false; + } + + @Override + public Segment getSegment(final DataSegment segment) throws SegmentLoadingException + { + return new SegmentForTesting( + MapUtils.getString(segment.getLoadSpec(), "version"), + (Interval) segment.getLoadSpec().get("interval") + ); + } + + @Override + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { + throw new UnsupportedOperationException(); + } + + @Override + public void cleanup(DataSegment segment) throws SegmentLoadingException + { + + } + }; + + private static class SegmentForTesting extends AbstractSegment + { + private final String version; + private final Interval interval; + private volatile boolean closed = false; + + SegmentForTesting( + String version, + Interval interval + ) + { + this.version = version; + this.interval = interval; + } + + public String getVersion() + { + return version; + } + + public Interval getInterval() + { + return interval; + } + + @Override + public String getIdentifier() + { + return version; + } + + public boolean isClosed() + { + return closed; + } + + @Override + public Interval getDataInterval() + { + return interval; + } + + @Override + public QueryableIndex asQueryableIndex() + { + throw new UnsupportedOperationException(); + } + + @Override + public StorageAdapter asStorageAdapter() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + closed = true; + } + } + + private static final List segments = ImmutableList.of( + new DataSegment( + "small_source", + new Interval("0/1000"), + "0", + ImmutableMap.of("interval", new Interval("0/1000"), "version", 0), + Lists.newArrayList(), + Lists.newArrayList(), + NoneShardSpec.instance(), + 0, + 10 + ), + new DataSegment( + "small_source", + new Interval("1000/2000"), + "0", + ImmutableMap.of("interval", new Interval("1000/2000"), "version", 0), + Lists.newArrayList(), + Lists.newArrayList(), + NoneShardSpec.instance(), + 0, + 10 + ), + new DataSegment( + "large_source", + new Interval("0/1000"), + "0", + ImmutableMap.of("interval", new Interval("0/1000"), "version", 0), + Lists.newArrayList(), + Lists.newArrayList(), + NoneShardSpec.instance(), + 0, + 100 + ), + new DataSegment( + "large_source", + new Interval("1000/2000"), + "0", + ImmutableMap.of("interval", new Interval("1000/2000"), "version", 0), + Lists.newArrayList(), + Lists.newArrayList(), + NoneShardSpec.instance(), + 0, + 100 + ), + // overshadowing the ahead segment + new DataSegment( + "large_source", + new Interval("1000/2000"), + "1", + ImmutableMap.of("interval", new Interval("1000/2000"), "version", 1), + Lists.newArrayList(), + Lists.newArrayList(), + NoneShardSpec.instance(), + 1, + 100 + ) + ); + + private ExecutorService executor; + private SegmentManager segmentManager; + + @Before + public void setup() + { + segmentManager = new SegmentManager(segmentLoader); + executor = Executors.newFixedThreadPool(segments.size()); + } + + @After + public void tearDown() + { + executor.shutdownNow(); + } + + @Test + public void testLoadSegment() throws ExecutionException, InterruptedException, SegmentLoadingException + { + final List> futures = segments.stream() + .map( + segment -> executor.submit( + () -> segmentManager.loadSegment(segment) + ) + ) + .collect(Collectors.toList()); + + for (Future eachFuture : futures) { + Assert.assertTrue(eachFuture.get()); + } + + assertResult(segments); + } + + @Test + public void testDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException + { + for (DataSegment eachSegment : segments) { + Assert.assertTrue(segmentManager.loadSegment(eachSegment)); + } + + final List> futures = ImmutableList.of(segments.get(0), segments.get(2)).stream() + .map( + segment -> executor.submit( + () -> { + segmentManager.dropSegment(segment); + return (Void) null; + } + ) + ) + .collect(Collectors.toList()); + + for (Future eachFuture : futures) { + eachFuture.get(); + } + + assertResult( + ImmutableList.of(segments.get(1), segments.get(3), segments.get(4)) + ); + } + + @Test + public void testLoadDropSegment() throws SegmentLoadingException, ExecutionException, InterruptedException + { + Assert.assertTrue(segmentManager.loadSegment(segments.get(0))); + Assert.assertTrue(segmentManager.loadSegment(segments.get(2))); + + final List> loadFutures = ImmutableList.of(segments.get(1), segments.get(3), segments.get(4)) + .stream() + .map( + segment -> executor.submit( + () -> segmentManager.loadSegment(segment) + ) + ) + .collect(Collectors.toList()); + final List> dropFutures = ImmutableList.of(segments.get(0), segments.get(2)).stream() + .map( + segment -> executor.submit( + () -> { + segmentManager.dropSegment(segment); + return (Void) null; + } + ) + ) + .collect(Collectors.toList()); + + for (Future eachFuture : loadFutures) { + Assert.assertTrue(eachFuture.get()); + } + for (Future eachFuture : dropFutures) { + eachFuture.get(); + } + + assertResult( + ImmutableList.of(segments.get(1), segments.get(3), segments.get(4)) + ); + } + + @Test + public void testLoadDuplicatedSegmentsSequentially() throws SegmentLoadingException + { + for (DataSegment segment : segments) { + Assert.assertTrue(segmentManager.loadSegment(segment)); + } + // try to load an existing segment + Assert.assertFalse(segmentManager.loadSegment(segments.get(0))); + + assertResult(segments); + } + + @Test + public void testLoadDuplicatedSegmentsInParallel() + throws ExecutionException, InterruptedException, SegmentLoadingException + { + final List> futures = ImmutableList.of(segments.get(0), segments.get(0), segments.get(0)).stream() + .map( + segment -> executor.submit( + () -> segmentManager.loadSegment(segment) + ) + ) + .collect(Collectors.toList()); + + int numSucceededFutures = 0; + int numFailedFutures = 0; + for (Future future : futures) { + numSucceededFutures += future.get() ? 1 : 0; + numFailedFutures += future.get() ? 0 : 1; + } + + Assert.assertEquals(1, numSucceededFutures); + Assert.assertEquals(2, numFailedFutures); + + assertResult(ImmutableList.of(segments.get(0))); + } + + @Test + public void testNonExistingSegmentsSequentially() throws SegmentLoadingException + { + Assert.assertTrue(segmentManager.loadSegment(segments.get(0))); + + // try to drop a non-existing segment of different data source + segmentManager.dropSegment(segments.get(2)); + assertResult( + ImmutableList.of(segments.get(0)) + ); + } + + @Test + public void testNonExistingSegmentsInParallel() + throws SegmentLoadingException, ExecutionException, InterruptedException + { + segmentManager.loadSegment(segments.get(0)); + final List> futures = ImmutableList.of(segments.get(1), segments.get(2)).stream() + .map( + segment -> executor.submit( + () -> { + segmentManager.dropSegment(segment); + return (Void) null; + } + ) + ) + .collect(Collectors.toList()); + + for (Future future : futures) { + future.get(); + } + + assertResult(ImmutableList.of(segments.get(0))); + } + + @Test + public void testRemoveEmptyTimeline() throws SegmentLoadingException + { + segmentManager.loadSegment(segments.get(0)); + assertResult(ImmutableList.of(segments.get(0))); + Assert.assertEquals(1, segmentManager.getDataSources().size()); + segmentManager.dropSegment(segments.get(0)); + Assert.assertEquals(0, segmentManager.getDataSources().size()); + } + + @Test + public void testGetNonExistingTimeline() + { + Assert.assertNull(segmentManager.getTimeline("nonExisting")); + } + + private void assertResult(List expectedExistingSegments) throws SegmentLoadingException + { + final Map expectedDataSourceSizes = expectedExistingSegments.stream() + .collect(Collectors.toMap( + DataSegment::getDataSource, + DataSegment::getSize, + Long::sum + )); + final Map expectedDataSourceCounts = expectedExistingSegments.stream() + .collect(Collectors.toMap( + DataSegment::getDataSource, + segment -> 1L, + Long::sum + )); + final Map> expectedDataSources + = new HashMap<>(); + for (DataSegment segment : expectedExistingSegments) { + final VersionedIntervalTimeline expectedTimeline = + expectedDataSources.computeIfAbsent( + segment.getDataSource(), + k -> new VersionedIntervalTimeline<>(Ordering.natural()) + ); + expectedTimeline.add( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk(new ReferenceCountingSegment(segmentLoader.getSegment(segment))) + ); + } + + Assert.assertEquals(expectedDataSourceCounts, segmentManager.getDataSourceCounts()); + Assert.assertEquals(expectedDataSourceSizes, segmentManager.getDataSourceSizes()); + + final Map dataSources = segmentManager.getDataSources(); + Assert.assertEquals(expectedDataSources.size(), dataSources.size()); + + dataSources.forEach( + (sourceName, dataSourceState) -> { + Assert.assertEquals(expectedDataSourceCounts.get(sourceName).longValue(), dataSourceState.getNumSegments()); + Assert.assertEquals(expectedDataSourceSizes.get(sourceName).longValue(), dataSourceState.getTotalSegmentSize()); + Assert.assertEquals( + expectedDataSources.get(sourceName).getAllTimelineEntries(), + dataSourceState.getTimeline().getAllTimelineEntries() + ); + } + ); + } +} diff --git a/server/src/test/java/io/druid/server/coordination/SegmentChangeRequestHistoryTest.java b/server/src/test/java/io/druid/server/coordination/SegmentChangeRequestHistoryTest.java new file mode 100644 index 000000000000..fe2c92306f7c --- /dev/null +++ b/server/src/test/java/io/druid/server/coordination/SegmentChangeRequestHistoryTest.java @@ -0,0 +1,246 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordination; + +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + */ +public class SegmentChangeRequestHistoryTest +{ + @Test + public void testSimple() throws Exception + { + SegmentChangeRequestHistory history = new SegmentChangeRequestHistory(); + Assert.assertEquals(0, history.getLastCounter().getCounter()); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + Assert.assertEquals(1, history.getLastCounter().getCounter()); + + SegmentChangeRequestsSnapshot snapshot = history.getRequestsSince(SegmentChangeRequestHistory.Counter.ZERO).get(); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(1, snapshot.getCounter().getCounter()); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + Assert.assertEquals(2, history.getLastCounter().getCounter()); + + snapshot = history.getRequestsSince(snapshot.getCounter()).get(); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + + snapshot = history.getRequestsSince(SegmentChangeRequestHistory.Counter.ZERO).get(); + Assert.assertEquals(2, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + } + + @Test + public void testTruncatedHistory() throws Exception + { + SegmentChangeRequestHistory history = new SegmentChangeRequestHistory(2); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter one = history.getLastCounter(); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter two = history.getLastCounter(); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter three = history.getLastCounter(); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter four = history.getLastCounter(); + + Assert.assertTrue(history.getRequestsSince(SegmentChangeRequestHistory.Counter.ZERO).get().isResetCounter()); + Assert.assertTrue(history.getRequestsSince(one).get().isResetCounter()); + Assert.assertTrue(history.getRequestsSince(two).get().isResetCounter()); + + SegmentChangeRequestsSnapshot snapshot = history.getRequestsSince(three).get(); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(4, snapshot.getCounter().getCounter()); + } + + @Test + public void testCounterHashMismatch() throws Exception + { + SegmentChangeRequestHistory history = new SegmentChangeRequestHistory(3); + + try { + history.getRequestsSince(new SegmentChangeRequestHistory.Counter(0, 1234)).get(); + Assert.fail(); + } catch (ExecutionException ex) { + Assert.assertTrue(ex.getCause() instanceof IllegalArgumentException); + } + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter one = history.getLastCounter(); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter two = history.getLastCounter(); + + try { + history.getRequestsSince(new SegmentChangeRequestHistory.Counter(0, 1234)).get(); + Assert.fail(); + } catch (ExecutionException ex) { + Assert.assertTrue(ex.getCause() instanceof IllegalArgumentException); + } + + SegmentChangeRequestsSnapshot snapshot = history.getRequestsSince(one).get(); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + + try { + history.getRequestsSince(new SegmentChangeRequestHistory.Counter(1, 1234)).get(); + Assert.fail(); + } catch (ExecutionException ex) { + Assert.assertTrue(ex.getCause() instanceof IllegalArgumentException); + } + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter three = history.getLastCounter(); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + SegmentChangeRequestHistory.Counter four = history.getLastCounter(); + + snapshot = history.getRequestsSince(two).get(); + Assert.assertEquals(2, snapshot.getRequests().size()); + Assert.assertEquals(4, snapshot.getCounter().getCounter()); + + try { + history.getRequestsSince(new SegmentChangeRequestHistory.Counter(2, 1234)).get(); + Assert.fail(); + } catch (ExecutionException ex) { + Assert.assertTrue(ex.getCause() instanceof IllegalArgumentException); + } + } + + @Test + public void testCancel() throws Exception + { + final SegmentChangeRequestHistory history = new SegmentChangeRequestHistory(); + + ListenableFuture future = history.getRequestsSince( + SegmentChangeRequestHistory.Counter.ZERO + ); + Assert.assertEquals(1, history.waitingFutures.size()); + + final AtomicBoolean callbackExcecuted = new AtomicBoolean(false); + Futures.addCallback( + future, + new FutureCallback() + { + @Override + public void onSuccess(SegmentChangeRequestsSnapshot result) + { + callbackExcecuted.set(true); + } + + @Override + public void onFailure(Throwable t) + { + callbackExcecuted.set(true); + } + } + ); + + future.cancel(true); + Assert.assertEquals(0, history.waitingFutures.size()); + Assert.assertFalse(callbackExcecuted.get()); + } + + @Test + public void testNonImmediateFuture() throws Exception + { + final SegmentChangeRequestHistory history = new SegmentChangeRequestHistory(); + + Future future = history.getRequestsSince( + SegmentChangeRequestHistory.Counter.ZERO + ); + + Assert.assertFalse(future.isDone()); + + history.addSegmentChangeRequest(new SegmentChangeRequestNoop()); + + SegmentChangeRequestsSnapshot snapshot = future.get(1, TimeUnit.MINUTES); + Assert.assertEquals(1, snapshot.getCounter().getCounter()); + Assert.assertEquals(1, snapshot.getRequests().size()); + } + + @Test + public void testCircularBuffer() throws Exception + { + SegmentChangeRequestHistory.CircularBuffer circularBuffer = new SegmentChangeRequestHistory.CircularBuffer<>( + 3); + + circularBuffer.add(1); + Assert.assertEquals(1, circularBuffer.size()); + Assert.assertEquals(1, (int) circularBuffer.get(0)); + + circularBuffer.add(2); + Assert.assertEquals(2, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+1, (int) circularBuffer.get(i)); + } + + circularBuffer.add(3); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+1, (int) circularBuffer.get(i)); + } + + circularBuffer.add(4); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+2, (int) circularBuffer.get(i)); + } + + circularBuffer.add(5); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+3, (int) circularBuffer.get(i)); + } + + circularBuffer.add(6); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+4, (int) circularBuffer.get(i)); + } + + circularBuffer.add(7); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+5, (int) circularBuffer.get(i)); + } + + circularBuffer.add(8); + Assert.assertEquals(3, circularBuffer.size()); + for (int i = 0; i < circularBuffer.size(); i++) { + Assert.assertEquals(i+6, (int) circularBuffer.get(i)); + } + } +} diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 7656e7583af9..2deda1ea5f2f 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -47,6 +47,7 @@ import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryMetrics; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -63,6 +64,7 @@ import io.druid.segment.StorageAdapter; import io.druid.segment.loading.SegmentLoader; import io.druid.segment.loading.SegmentLoadingException; +import io.druid.server.SegmentManager; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -94,6 +96,7 @@ public class ServerManagerTest private CountDownLatch queryWaitYieldLatch; private CountDownLatch queryNotifyLatch; private ExecutorService serverManagerExec; + private SegmentManager segmentManager; @Before public void setUp() throws IOException @@ -105,7 +108,7 @@ public void setUp() throws IOException queryNotifyLatch = new CountDownLatch(1); factory = new MyQueryRunnerFactory(queryWaitLatch, queryWaitYieldLatch, queryNotifyLatch); serverManagerExec = Executors.newFixedThreadPool(2); - serverManager = new ServerManager( + segmentManager = new SegmentManager( new SegmentLoader() { @Override @@ -134,7 +137,9 @@ public void cleanup(DataSegment segment) throws SegmentLoadingException { } - }, + } + ); + serverManager = new ServerManager( new QueryRunnerFactoryConglomerate() { @Override @@ -148,7 +153,8 @@ public > QueryRunnerFactory findFact MoreExecutors.sameThreadExecutor(), new DefaultObjectMapper(), new LocalCacheProvider().get(), - new CacheConfig() + new CacheConfig(), + segmentManager ); loadQueryable("test", "1", new Interval("P1d/2011-04-01")); @@ -458,7 +464,7 @@ public void run() public void loadQueryable(String dataSource, String version, Interval interval) throws IOException { try { - serverManager.loadSegment( + segmentManager.loadSegment( new DataSegment( dataSource, interval, @@ -480,7 +486,7 @@ public void loadQueryable(String dataSource, String version, Interval interval) public void dropQueryable(String dataSource, String version, Interval interval) { try { - serverManager.dropSegment( + segmentManager.dropSegment( new DataSegment( dataSource, interval, @@ -679,9 +685,9 @@ public BlockingQueryRunner( } @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { - return new BlockingSequence(runner.run(query, responseContext), waitLatch, waitYieldLatch, notifyLatch); + return new BlockingSequence<>(runner.run(queryPlus, responseContext), waitLatch, waitYieldLatch, notifyLatch); } } diff --git a/server/src/test/java/io/druid/server/coordination/ServerTypeTest.java b/server/src/test/java/io/druid/server/coordination/ServerTypeTest.java new file mode 100644 index 000000000000..27c63e6bc7c6 --- /dev/null +++ b/server/src/test/java/io/druid/server/coordination/ServerTypeTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordination; + +import org.junit.Assert; +import org.junit.Test; + +public class ServerTypeTest +{ + @Test + public void testAssignable() + { + Assert.assertTrue(ServerType.HISTORICAL.isSegmentReplicationTarget()); + Assert.assertTrue(ServerType.BRIDGE.isSegmentReplicationTarget()); + Assert.assertFalse(ServerType.REALTIME.isSegmentReplicationTarget()); + Assert.assertFalse(ServerType.INDEXER_EXECUTOR.isSegmentReplicationTarget()); + } + + @Test + public void testFromString() + { + Assert.assertEquals(ServerType.HISTORICAL, ServerType.fromString("historical")); + Assert.assertEquals(ServerType.BRIDGE, ServerType.fromString("bridge")); + Assert.assertEquals(ServerType.REALTIME, ServerType.fromString("realtime")); + Assert.assertEquals(ServerType.INDEXER_EXECUTOR, ServerType.fromString("indexer-executor")); + } + + @Test + public void testToString() + { + Assert.assertEquals(ServerType.HISTORICAL.toString(), "historical"); + Assert.assertEquals(ServerType.BRIDGE.toString(), "bridge"); + Assert.assertEquals(ServerType.REALTIME.toString(), "realtime"); + Assert.assertEquals(ServerType.INDEXER_EXECUTOR.toString(), "indexer-executor"); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidName() + { + ServerType.fromString("invalid"); + } +} diff --git a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java index bfafb38c14c1..d8233cbff2bf 100644 --- a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java @@ -29,7 +29,6 @@ import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Module; - import io.druid.client.cache.CacheConfig; import io.druid.client.cache.LocalCacheProvider; import io.druid.concurrent.Execs; @@ -44,12 +43,14 @@ import io.druid.segment.IndexIO; import io.druid.segment.loading.CacheTestSegmentLoader; import io.druid.segment.loading.SegmentLoaderConfig; +import io.druid.server.SegmentManager; import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.apache.curator.framework.CuratorFramework; +import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -81,7 +82,7 @@ public class ZkCoordinatorTest extends CuratorTestBase "dummyServer", "dummyHost", 0, - "dummyType", + ServerType.HISTORICAL, "normal", 0 ); @@ -93,6 +94,7 @@ public class ZkCoordinatorTest extends CuratorTestBase private AtomicInteger announceCount; private ConcurrentSkipListSet segmentsAnnouncedByMe; private CacheTestSegmentLoader segmentLoader; + private SegmentManager segmentManager; private List scheduledRunnable; @Before @@ -116,16 +118,17 @@ public void setUp() throws Exception scheduledRunnable = Lists.newArrayList(); segmentLoader = new CacheTestSegmentLoader(); + segmentManager = new SegmentManager(segmentLoader); serverManager = new ServerManager( - segmentLoader, new NoopQueryRunnerFactoryConglomerate(), new NoopServiceEmitter(), MoreExecutors.sameThreadExecutor(), MoreExecutors.sameThreadExecutor(), new DefaultObjectMapper(), new LocalCacheProvider().get(), - new CacheConfig() + new CacheConfig(), + segmentManager ); final ZkPathsConfig zkPaths = new ZkPathsConfig() @@ -139,6 +142,7 @@ public String getBase() segmentsAnnouncedByMe = new ConcurrentSkipListSet<>(); announceCount = new AtomicInteger(0); + announcer = new DataSegmentAnnouncer() { private final DataSegmentAnnouncer delegate = new BatchDataSegmentAnnouncer( @@ -184,12 +188,6 @@ public void unannounceSegments(Iterable segments) throws IOExceptio announceCount.addAndGet(-Iterables.size(segments)); delegate.unannounceSegments(segments); } - - @Override - public boolean isAnnounced(DataSegment segment) - { - return segmentsAnnouncedByMe.contains(segment); - } }; zkCoordinator = new ZkCoordinator( @@ -223,8 +221,9 @@ public int getDropSegmentDelayMillis() zkPaths, me, announcer, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), curator, - serverManager, + segmentManager, new ScheduledExecutorFactory() { @Override @@ -392,12 +391,12 @@ public void testLoadCache() throws Exception } checkCache(segments); - Assert.assertTrue(serverManager.getDataSourceCounts().isEmpty()); + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); zkCoordinator.start(); - Assert.assertTrue(!serverManager.getDataSourceCounts().isEmpty()); + Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { - Assert.assertEquals(11L, serverManager.getDataSourceCounts().get("test" + i).longValue()); - Assert.assertEquals(2L, serverManager.getDataSourceCounts().get("test_two" + i).longValue()); + Assert.assertEquals(11L, segmentManager.getDataSourceCounts().get("test" + i).longValue()); + Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } Assert.assertEquals(13 * COUNT, announceCount.get()); zkCoordinator.stop(); @@ -516,10 +515,12 @@ public String getBase() } ); binder.bind(DruidServerMetadata.class) - .toInstance(new DruidServerMetadata("dummyServer", "dummyHost", 0, "dummyType", "normal", 0)); + .toInstance(new DruidServerMetadata("dummyServer", "dummyHost", 0, ServerType.HISTORICAL, "normal", 0)); binder.bind(DataSegmentAnnouncer.class).toInstance(announcer); + binder.bind(DataSegmentServerAnnouncer.class).toInstance(EasyMock.createNiceMock(DataSegmentServerAnnouncer.class)); binder.bind(CuratorFramework.class).toInstance(curator); binder.bind(ServerManager.class).toInstance(serverManager); + binder.bind(SegmentManager.class).toInstance(segmentManager); binder.bind(ScheduledExecutorFactory.class).toInstance(ScheduledExecutors.createFactory(new Lifecycle())); } @@ -543,13 +544,13 @@ public String getBase() } checkCache(segments); - Assert.assertTrue(serverManager.getDataSourceCounts().isEmpty()); + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); zkCoordinator.start(); - Assert.assertTrue(!serverManager.getDataSourceCounts().isEmpty()); + Assert.assertTrue(!segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { - Assert.assertEquals(3L, serverManager.getDataSourceCounts().get("test" + i).longValue()); - Assert.assertEquals(2L, serverManager.getDataSourceCounts().get("test_two" + i).longValue()); + Assert.assertEquals(3L, segmentManager.getDataSourceCounts().get("test" + i).longValue()); + Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); } Assert.assertEquals(5 * COUNT, announceCount.get()); zkCoordinator.stop(); diff --git a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 359631aa8322..6790dad3cd9d 100644 --- a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -33,6 +33,9 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.SegmentChangeRequestHistory; +import io.druid.server.coordination.SegmentChangeRequestsSnapshot; +import io.druid.server.coordination.ServerType; import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; @@ -104,7 +107,7 @@ public void setUp() throws Exception "id", "host", Long.MAX_VALUE, - "type", + ServerType.HISTORICAL, "tier", 0 ), @@ -145,7 +148,6 @@ public String getBase() announcer, jsonMapper ); - segmentAnnouncer.start(); testSegments = Sets.newHashSet(); for (int i = 0; i < 100; i++) { @@ -156,7 +158,6 @@ public String getBase() @After public void tearDown() throws Exception { - segmentAnnouncer.stop(); announcer.stop(); cf.close(); testingCluster.stop(); @@ -185,6 +186,12 @@ public void testSingleAnnounce() throws Exception Assert.assertEquals(Sets.newHashSet(firstSegment, secondSegment), segments); } + SegmentChangeRequestsSnapshot snapshot = segmentAnnouncer.getSegmentChangesSince( + new SegmentChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals(2, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + segmentAnnouncer.unannounceSegment(firstSegment); for (String zNode : zNodes) { @@ -195,6 +202,18 @@ public void testSingleAnnounce() throws Exception segmentAnnouncer.unannounceSegment(secondSegment); Assert.assertTrue(cf.getChildren().forPath(testSegmentsPath).isEmpty()); + + snapshot = segmentAnnouncer.getSegmentChangesSince( + snapshot.getCounter() + ).get(); + Assert.assertEquals(2, snapshot.getRequests().size()); + Assert.assertEquals(4, snapshot.getCounter().getCounter()); + + snapshot = segmentAnnouncer.getSegmentChangesSince( + new SegmentChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals(0, snapshot.getRequests().size()); + Assert.assertEquals(4, snapshot.getCounter().getCounter()); } @Test @@ -272,6 +291,11 @@ public void testSingleAnnounceManyTimes() throws Exception @Test public void testBatchAnnounce() throws Exception + { + testBatchAnnounce(true); + } + + private void testBatchAnnounce(boolean testHistory) throws Exception { segmentAnnouncer.announceSegments(testSegments); @@ -285,16 +309,40 @@ public void testBatchAnnounce() throws Exception } Assert.assertEquals(allSegments, testSegments); + SegmentChangeRequestsSnapshot snapshot = null; + + if (testHistory) { + snapshot = segmentAnnouncer.getSegmentChangesSince( + new SegmentChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals(testSegments.size(), snapshot.getRequests().size()); + Assert.assertEquals(testSegments.size(), snapshot.getCounter().getCounter()); + } + segmentAnnouncer.unannounceSegments(testSegments); Assert.assertTrue(cf.getChildren().forPath(testSegmentsPath).isEmpty()); + + if (testHistory) { + snapshot = segmentAnnouncer.getSegmentChangesSince( + snapshot.getCounter() + ).get(); + Assert.assertEquals(testSegments.size(), snapshot.getRequests().size()); + Assert.assertEquals(2 * testSegments.size(), snapshot.getCounter().getCounter()); + + snapshot = segmentAnnouncer.getSegmentChangesSince( + new SegmentChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals(0, snapshot.getRequests().size()); + Assert.assertEquals(2 * testSegments.size(), snapshot.getCounter().getCounter()); + } } @Test public void testMultipleBatchAnnounce() throws Exception { for (int i = 0; i < 10; i++) { - testBatchAnnounce(); + testBatchAnnounce(false); } } @@ -315,7 +363,7 @@ private DataSegment makeSegment(int offset) .build(); } - private class SegmentReader + private static class SegmentReader { private final CuratorFramework cf; private final ObjectMapper jsonMapper; diff --git a/server/src/test/java/io/druid/server/coordinator/CoordinatorStatsTest.java b/server/src/test/java/io/druid/server/coordinator/CoordinatorStatsTest.java new file mode 100644 index 000000000000..ea33c5301f49 --- /dev/null +++ b/server/src/test/java/io/druid/server/coordinator/CoordinatorStatsTest.java @@ -0,0 +1,136 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordinator; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Map; + +public class CoordinatorStatsTest +{ + private CoordinatorStats stats; + + @Before + public void setUp() throws Exception + { + stats = new CoordinatorStats(); + } + + @After + public void tearDown() throws Exception + { + stats = null; + } + + @Test + public void addToGlobalStat() throws Exception + { + Assert.assertEquals(0, stats.getGlobalStat("stats")); + stats.addToGlobalStat("stats", 1); + Assert.assertEquals(1, stats.getGlobalStat("stats")); + stats.addToGlobalStat("stats", -11); + Assert.assertEquals(-10, stats.getGlobalStat("stats")); + } + + @Test(expected = NullPointerException.class) + public void testAddToTieredStatNonexistentStat() throws Exception + { + stats.getTieredStat("stat", "tier"); + } + + @Test + public void testAddToTieredStat() throws Exception + { + Assert.assertFalse(stats.hasPerTierStats()); + stats.addToTieredStat("stat1", "tier1", 1); + stats.addToTieredStat("stat1", "tier2", 1); + stats.addToTieredStat("stat1", "tier1", -5); + stats.addToTieredStat("stat2", "tier1", 1); + stats.addToTieredStat("stat1", "tier2", 1); + Assert.assertTrue(stats.hasPerTierStats()); + + Assert.assertEquals( + Sets.newHashSet("tier1", "tier2"), + stats.getTiers("stat1") + ); + Assert.assertEquals( + Sets.newHashSet("tier1"), + stats.getTiers("stat2") + ); + Assert.assertTrue(stats.getTiers("stat3").isEmpty()); + + Assert.assertEquals(-4, stats.getTieredStat("stat1", "tier1")); + Assert.assertEquals(2, stats.getTieredStat("stat1", "tier2")); + Assert.assertEquals(1, stats.getTieredStat("stat2", "tier1")); + } + + @Test + public void testForEachTieredStat() throws Exception + { + final Map expected = ImmutableMap.of( + "tier1", 1L, + "tier2", 2L, + "tier3", 3L + ); + final Map actual = Maps.newHashMap(); + + expected.forEach( + (tier, count) -> stats.addToTieredStat("stat", tier, count) + ); + + stats.forEachTieredStat("stat0", (tier, count) -> Assert.fail()); + stats.forEachTieredStat("stat", actual::put); + + Assert.assertEquals(expected, actual); + } + + + @Test + public void testAccumulate() throws Exception + { + stats.addToGlobalStat("stat1", 1); + stats.addToGlobalStat("stat2", 1); + stats.addToTieredStat("stat1", "tier1", 1); + stats.addToTieredStat("stat1", "tier2", 1); + stats.addToTieredStat("stat2", "tier1", 1); + + final CoordinatorStats stats2 = new CoordinatorStats(); + stats2.addToGlobalStat("stat1", 1); + stats2.addToTieredStat("stat1", "tier2", 1); + stats2.addToTieredStat("stat2", "tier2", 1); + stats2.addToTieredStat("stat3", "tier1", 1); + + stats.accumulate(stats2); + + Assert.assertEquals(2, stats.getGlobalStat("stat1")); + Assert.assertEquals(1, stats.getGlobalStat("stat2")); + Assert.assertEquals(1, stats.getTieredStat("stat1", "tier1")); + Assert.assertEquals(2, stats.getTieredStat("stat1", "tier2")); + Assert.assertEquals(1, stats.getTieredStat("stat2", "tier1")); + Assert.assertEquals(1, stats.getTieredStat("stat2", "tier2")); + Assert.assertEquals(1, stats.getTieredStat("stat3", "tier1")); + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyBenchmark.java b/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyBenchmark.java index d1c0ac16513d..874b6a745f88 100644 --- a/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyBenchmark.java +++ b/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyBenchmark.java @@ -45,11 +45,11 @@ public static List factoryClasses() return Arrays.asList( (CostBalancerStrategy[]) Arrays.asList( new CostBalancerStrategy(MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1))) + Executors.newFixedThreadPool(1))) ).toArray(), (CostBalancerStrategy[]) Arrays.asList( new CostBalancerStrategy(MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(4))) + Executors.newFixedThreadPool(4))) ).toArray() ); } @@ -71,7 +71,8 @@ public static void setup() } @AfterClass - public static void tearDown(){ + public static void tearDown() + { serverHolderList = null; } diff --git a/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyTest.java b/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyTest.java index 2b4a098a79cc..e1bff13b2874 100644 --- a/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyTest.java +++ b/server/src/test/java/io/druid/server/coordinator/CostBalancerStrategyTest.java @@ -26,6 +26,7 @@ import io.druid.client.ImmutableDruidDataSource; import io.druid.client.ImmutableDruidServer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -61,7 +62,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme serverHolderList.add( new ServerHolder( new ImmutableDruidServer( - new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, "hot", "hot", 1), + new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, ServerType.HISTORICAL, "hot", 1), 3000L, ImmutableMap.of("DUMMY", EasyMock.createMock(ImmutableDruidDataSource.class)), ImmutableMap.copyOf(segments) diff --git a/server/src/test/java/io/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java b/server/src/test/java/io/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java index b869c83ae1d1..f6b0267162e4 100644 --- a/server/src/test/java/io/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java @@ -26,6 +26,7 @@ import io.druid.client.ImmutableDruidDataSource; import io.druid.client.ImmutableDruidServer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -60,7 +61,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme serverHolderList.add( new ServerHolder( new ImmutableDruidServer( - new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, "hot", "hot", 1), + new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, ServerType.HISTORICAL, "hot", 1), 3000L, ImmutableMap.of("DUMMY", EasyMock.createMock(ImmutableDruidDataSource.class)), ImmutableMap.copyOf(segments) diff --git a/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java b/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java new file mode 100644 index 000000000000..d91675f9e42f --- /dev/null +++ b/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordinator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.MinMaxPriorityQueue; +import com.google.common.collect.Ordering; +import io.druid.client.ImmutableDruidDataSource; +import io.druid.client.ImmutableDruidServer; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DruidClusterTest +{ + private static final List segments = ImmutableList.of( + new DataSegment( + "test", + new Interval("2015-04-12/2015-04-13"), + "1", + ImmutableMap.of("containerName", "container1", "blobPath", "blobPath1"), + null, + null, + NoneShardSpec.instance(), + 0, + 1 + ), + new DataSegment( + "test", + new Interval("2015-04-12/2015-04-13"), + "1", + ImmutableMap.of("containerName", "container2", "blobPath", "blobPath2"), + null, + null, + NoneShardSpec.instance(), + 0, + 1 + ) + ); + + private static final Map dataSources = ImmutableMap.of( + "src1", + new ImmutableDruidDataSource( + "src1", + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableSet.of() + ), + "src2", + new ImmutableDruidDataSource( + "src2", + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableSet.of() + ) + ); + + private static final ServerHolder newRealtime = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host2", 100L, ServerType.REALTIME, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + private static final ServerHolder newHistorical = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host2", 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + private DruidCluster cluster; + + @Before + public void setup() + { + cluster = new DruidCluster( + ImmutableSet.of( + new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 100L, ServerType.REALTIME, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ) + ), + ImmutableMap.of( + "tier1", + MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( + ImmutableList.of( + new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ) + ) + ) + ) + ); + } + + @Test + public void testAdd() + { + Assert.assertEquals(1, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum()); + Assert.assertEquals(1, cluster.getRealtimes().size()); + + cluster.add(newRealtime); + Assert.assertEquals(1, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum()); + Assert.assertEquals(2, cluster.getRealtimes().size()); + + cluster.add(newHistorical); + Assert.assertEquals(2, cluster.getHistoricals().values().stream().mapToInt(Collection::size).sum()); + Assert.assertEquals(2, cluster.getRealtimes().size()); + } + + @Test + public void testGetAllServers() + { + cluster.add(newRealtime); + cluster.add(newHistorical); + final Collection allServers = cluster.getAllServers(); + Assert.assertEquals(4, allServers.size()); + Assert.assertTrue(allServers.containsAll(cluster.getRealtimes())); + Assert.assertTrue( + allServers.containsAll( + cluster.getHistoricals().values().stream().flatMap(Collection::stream).collect(Collectors.toList()) + ) + ); + } + + @Test + public void testIsEmpty() + { + final DruidCluster emptyCluster = new DruidCluster(); + Assert.assertFalse(cluster.isEmpty()); + Assert.assertTrue(emptyCluster.isEmpty()); + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index 523a7b08e3f3..afda7460561a 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -134,6 +134,7 @@ public void bigProfiler() DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( + null, ImmutableMap.>of( "normal", MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) @@ -161,6 +162,7 @@ public void bigProfiler() .withSegmentReplicantLookup( SegmentReplicantLookup.make( new DruidCluster( + null, ImmutableMap.>of( "normal", MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) @@ -216,6 +218,7 @@ public void profileRun() DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( + null, ImmutableMap.>of( "normal", MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java index 29ca9b4d33c0..557dc3cb8aa3 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -179,6 +179,7 @@ public void testMoveToEmptyServerBalancer() throws IOException DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( + null, ImmutableMap.>of( "normal", MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) @@ -210,8 +211,8 @@ public void testMoveToEmptyServerBalancer() throws IOException .build(); params = new DruidCoordinatorBalancerTester(coordinator).run(params); - Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() > 0); - Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() < segments.size()); + Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0); + Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") < segments.size()); exec.shutdown(); } @@ -260,6 +261,7 @@ public void testRun1() throws IOException DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( + null, ImmutableMap.>of( "normal", MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) @@ -290,7 +292,7 @@ public void testRun1() throws IOException .build(); params = new DruidCoordinatorBalancerTester(coordinator).run(params); - Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0); exec.shutdown(); } @@ -354,6 +356,7 @@ public void testRun2() throws IOException DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( + null, ImmutableMap.>of( "normal", MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) @@ -391,7 +394,7 @@ public void testRun2() throws IOException .build(); params = new DruidCoordinatorBalancerTester(coordinator).run(params); - Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getCoordinatorStats().getTieredStat("movedCount", "normal") > 0); exec.shutdown(); } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java index 4af21b425c75..4327d8720ecb 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java @@ -46,7 +46,6 @@ protected void moveSegment( if (!toPeon.getSegmentsToLoad().contains(segmentToMove) && !currentlyMovingSegments.get("normal").containsKey(segmentName) && - !toServer.getSegments().containsKey(segmentName) && new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) { log.info( "Moving [%s] from [%s] to [%s]", @@ -71,7 +70,7 @@ public void execute() log.info(e, String.format("[%s] : Moving exception", segmentName)); } } else { - currentlyMovingSegments.get("normal").remove(segment); + currentlyMovingSegments.get("normal").remove(segmentName); } } } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index f82cb5eb8507..f59fbdcb246e 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -33,6 +33,7 @@ import io.druid.client.DruidServer; import io.druid.metadata.MetadataRuleManager; import io.druid.segment.IndexIO; +import io.druid.server.coordination.ServerType; import io.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; import io.druid.server.coordinator.rules.ForeverLoadRule; import io.druid.server.coordinator.rules.IntervalDropRule; @@ -130,6 +131,7 @@ public void testRunThreeTiersOneReplicant() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -139,7 +141,7 @@ public void testRunThreeTiersOneReplicant() throws Exception "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -155,7 +157,7 @@ public void testRunThreeTiersOneReplicant() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ).toImmutableDruidServer(), @@ -171,7 +173,7 @@ public void testRunThreeTiersOneReplicant() throws Exception "serverCold", "hostCold", 1000, - "historical", + ServerType.HISTORICAL, "cold", 0 ).toImmutableDruidServer(), @@ -201,11 +203,11 @@ public void testRunThreeTiersOneReplicant() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 6); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("normal").get() == 6); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("cold").get() == 12); - Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); - Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + Assert.assertEquals(6L, stats.getTieredStat("assignedCount", "hot")); + Assert.assertEquals(6L, stats.getTieredStat("assignedCount", "normal")); + Assert.assertEquals(12L, stats.getTieredStat("assignedCount", "cold")); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); exec.shutdown(); EasyMock.verify(mockPeon); @@ -237,6 +239,7 @@ public void testRunTwoTiersTwoReplicants() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -246,7 +249,7 @@ public void testRunTwoTiersTwoReplicants() throws Exception "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -257,7 +260,7 @@ public void testRunTwoTiersTwoReplicants() throws Exception "serverHot2", "hostHot2", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -273,7 +276,7 @@ public void testRunTwoTiersTwoReplicants() throws Exception "serverCold", "hostCold", 1000, - "historical", + ServerType.HISTORICAL, "cold", 0 ).toImmutableDruidServer(), @@ -302,10 +305,10 @@ public void testRunTwoTiersTwoReplicants() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 12); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("cold").get() == 18); - Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); - Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + Assert.assertEquals(12L, stats.getTieredStat("assignedCount", "hot")); + Assert.assertEquals(18L, stats.getTieredStat("assignedCount", "cold")); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); exec.shutdown(); EasyMock.verify(mockPeon); @@ -340,7 +343,7 @@ public void testRunTwoTiersWithExistingSegments() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -349,6 +352,7 @@ public void testRunTwoTiersWithExistingSegments() throws Exception } DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -358,7 +362,7 @@ public void testRunTwoTiersWithExistingSegments() throws Exception "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -398,10 +402,10 @@ public void testRunTwoTiersWithExistingSegments() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 12); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("normal").get() == 0); - Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); - Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + Assert.assertEquals(12L, stats.getTieredStat("assignedCount", "hot")); + Assert.assertEquals(0L, stats.getTieredStat("assignedCount", "normal")); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); exec.shutdown(); EasyMock.verify(mockPeon); @@ -430,6 +434,7 @@ public void testRunTwoTiersTierDoesNotExist() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -439,7 +444,7 @@ public void testRunTwoTiersTierDoesNotExist() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ).toImmutableDruidServer(), @@ -489,6 +494,7 @@ public void testRunRuleDoesNotExist() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -498,7 +504,7 @@ public void testRunRuleDoesNotExist() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ).toImmutableDruidServer(), @@ -553,7 +559,7 @@ public void testDropRemove() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -562,6 +568,7 @@ public void testDropRemove() throws Exception } DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -595,7 +602,7 @@ public void testDropRemove() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); + Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); exec.shutdown(); EasyMock.verify(coordinator); @@ -623,7 +630,7 @@ public void testDropTooManyInSameTier() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -633,7 +640,7 @@ public void testDropTooManyInSameTier() throws Exception "serverNorm2", "hostNorm2", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -642,6 +649,7 @@ public void testDropTooManyInSameTier() throws Exception } DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -679,8 +687,8 @@ public void testDropTooManyInSameTier() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 1); - Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "normal")); + Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); exec.shutdown(); EasyMock.verify(mockPeon); @@ -710,7 +718,7 @@ public void testDropTooManyInDifferentTiers() throws Exception "server1", "host1", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ); @@ -719,7 +727,7 @@ public void testDropTooManyInDifferentTiers() throws Exception "serverNorm2", "hostNorm2", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -728,6 +736,7 @@ public void testDropTooManyInDifferentTiers() throws Exception } DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -770,8 +779,8 @@ public void testDropTooManyInDifferentTiers() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 1); - Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "normal")); + Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); exec.shutdown(); EasyMock.verify(mockPeon); @@ -799,7 +808,7 @@ public void testDontDropInDifferentTiers() throws Exception "server1", "host1", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ); @@ -807,7 +816,7 @@ public void testDontDropInDifferentTiers() throws Exception "serverNorm2", "hostNorm2", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -815,6 +824,7 @@ public void testDontDropInDifferentTiers() throws Exception server2.addDataSegment(segment.getIdentifier(), segment); } DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -857,8 +867,8 @@ public void testDontDropInDifferentTiers() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount") == null); - Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); + Assert.assertTrue(stats.getTiers("droppedCount").isEmpty()); + Assert.assertEquals(12L, stats.getGlobalStat("deletedCount")); exec.shutdown(); EasyMock.verify(mockPeon); @@ -879,7 +889,7 @@ public void testDropServerActuallyServesSegment() throws Exception "server1", "host1", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -888,7 +898,7 @@ public void testDropServerActuallyServesSegment() throws Exception "serverNorm2", "hostNorm2", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -897,7 +907,7 @@ public void testDropServerActuallyServesSegment() throws Exception "serverNorm3", "hostNorm3", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -916,6 +926,7 @@ public void testDropServerActuallyServesSegment() throws Exception EasyMock.replay(anotherMockPeon); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -957,7 +968,7 @@ public void testDropServerActuallyServesSegment() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 1); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "normal")); exec.shutdown(); EasyMock.verify(mockPeon); @@ -988,6 +999,7 @@ public void testReplicantThrottle() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -997,7 +1009,7 @@ public void testReplicantThrottle() throws Exception "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -1008,7 +1020,7 @@ public void testReplicantThrottle() throws Exception "serverHot2", "hostHot2", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -1037,9 +1049,9 @@ public void testReplicantThrottle() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 48); - Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); - Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + Assert.assertEquals(48L, stats.getTieredStat("assignedCount", "hot")); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); DataSegment overFlowSegment = new DataSegment( "test", @@ -1066,9 +1078,9 @@ public void testReplicantThrottle() throws Exception ); stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 1); - Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); - Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + Assert.assertEquals(1L, stats.getTieredStat("assignedCount", "hot")); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); EasyMock.verify(mockPeon); exec.shutdown(); @@ -1112,6 +1124,7 @@ public void testReplicantThrottleAcrossTiers() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -1121,7 +1134,7 @@ public void testReplicantThrottleAcrossTiers() throws Exception "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -1137,7 +1150,7 @@ public void testReplicantThrottleAcrossTiers() throws Exception "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ).toImmutableDruidServer(), @@ -1167,10 +1180,10 @@ public void testReplicantThrottleAcrossTiers() throws Exception DruidCoordinatorRuntimeParams afterParams = runner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 24); - Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get(DruidServer.DEFAULT_TIER).get() == 7); - Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); - Assert.assertTrue(stats.getPerTierStats().get("unassignedSize") == null); + Assert.assertEquals(24L, stats.getTieredStat("assignedCount", "hot")); + Assert.assertEquals(7L, stats.getTieredStat("assignedCount", DruidServer.DEFAULT_TIER)); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); EasyMock.verify(mockPeon); exec.shutdown(); @@ -1211,7 +1224,7 @@ public void testDropReplicantThrottle() throws Exception "serverNorm1", "hostNorm1", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -1222,7 +1235,7 @@ public void testDropReplicantThrottle() throws Exception "serverNorm2", "hostNorm2", 1000, - "historical", + ServerType.HISTORICAL, "normal", 0 ); @@ -1231,6 +1244,7 @@ public void testDropReplicantThrottle() throws Exception } DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -1269,7 +1283,7 @@ public void testDropReplicantThrottle() throws Exception CoordinatorStats stats = afterParams.getCoordinatorStats(); // There is no throttling on drop - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 25); + Assert.assertEquals(25L, stats.getTieredStat("droppedCount", "normal")); EasyMock.verify(mockPeon); exec.shutdown(); } @@ -1317,6 +1331,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() throws Exception EasyMock.replay(databaseRuleManager); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( DruidServer.DEFAULT_TIER, MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -1326,7 +1341,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() throws Exception "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ).toImmutableDruidServer(), @@ -1356,10 +1371,10 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertEquals(1, stats.getPerTierStats().get("assignedCount").size()); - Assert.assertEquals(1, stats.getPerTierStats().get("assignedCount").get("_default_tier").get()); - Assert.assertNull(stats.getPerTierStats().get("unassignedCount")); - Assert.assertNull(stats.getPerTierStats().get("unassignedSize")); + Assert.assertEquals(1, stats.getTiers("assignedCount").size()); + Assert.assertEquals(1, stats.getTieredStat("assignedCount", "_default_tier")); + Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); + Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); Assert.assertEquals(2, availableSegments.size()); Assert.assertEquals(availableSegments, params.getAvailableSegments()); diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index 18ca9f423d05..4d8fdb7f08a6 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -31,24 +31,24 @@ import io.druid.client.ImmutableDruidDataSource; import io.druid.client.ImmutableDruidServer; import io.druid.client.SingleServerInventoryView; -import io.druid.collections.CountingMap; import io.druid.common.config.JacksonConfigManager; import io.druid.concurrent.Execs; import io.druid.curator.CuratorTestBase; import io.druid.curator.discovery.NoopServiceAnnouncer; -import io.druid.curator.inventory.InventoryManagerConfig; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.metadata.MetadataRuleManager; import io.druid.metadata.MetadataSegmentManager; import io.druid.server.DruidNode; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.server.coordinator.rules.ForeverLoadRule; import io.druid.server.coordinator.rules.Rule; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; @@ -216,7 +216,7 @@ public void testMoveSegment() throws Exception EasyMock.replay(metadataRuleManager); EasyMock.expect(druidServer.toImmutableDruidServer()).andReturn( new ImmutableDruidServer( - new DruidServerMetadata("from", null, 5L, null, null, 0), + new DruidServerMetadata("from", null, 5L, ServerType.HISTORICAL, null, 0), 1L, null, ImmutableMap.of("dummySegment", segment) @@ -227,7 +227,7 @@ public void testMoveSegment() throws Exception druidServer2 = EasyMock.createMock(DruidServer.class); EasyMock.expect(druidServer2.toImmutableDruidServer()).andReturn( new ImmutableDruidServer( - new DruidServerMetadata("to", null, 5L, null, null, 0), + new DruidServerMetadata("to", null, 5L, ServerType.HISTORICAL, null, 0), 1L, null, ImmutableMap.of("dummySegment2", segment) @@ -238,22 +238,6 @@ public void testMoveSegment() throws Exception loadManagementPeons.put("from", loadQueuePeon); loadManagementPeons.put("to", loadQueuePeon); - EasyMock.expect(serverInventoryView.getInventoryManagerConfig()).andReturn( - new InventoryManagerConfig() - { - @Override - public String getContainerPath() - { - return ""; - } - - @Override - public String getInventoryPath() - { - return ""; - } - } - ); EasyMock.replay(serverInventoryView); coordinator.moveSegment( @@ -301,7 +285,7 @@ public void testCoordinatorRun() throws Exception{ EasyMock.replay(immutableDruidDataSource); // Setup ServerInventoryView - druidServer = new DruidServer("server1", "localhost", 5L, "historical", tier, 0); + druidServer = new DruidServer("server1", "localhost", 5L, ServerType.HISTORICAL, tier, 0); loadManagementPeons.put("server1", loadQueuePeon); EasyMock.expect(serverInventoryView.getInventory()).andReturn( ImmutableList.of(druidServer) @@ -346,14 +330,14 @@ public void childEvent( Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getIdentifier())); // Wait for coordinator thread to run so that replication status is updated - while (coordinator.getSegmentAvailability().snapshot().get(dataSource) != 0) { + while (coordinator.getSegmentAvailability().getLong(dataSource) != 0) { Thread.sleep(50); } - Map segmentAvailability = coordinator.getSegmentAvailability().snapshot(); + Map segmentAvailability = coordinator.getSegmentAvailability(); Assert.assertEquals(1, segmentAvailability.size()); Assert.assertEquals(0L, segmentAvailability.get(dataSource)); - while (coordinator.getLoadPendingDatasources().get(dataSource).get() > 0) { + while (coordinator.hasLoadPending(dataSource)) { Thread.sleep(50); } @@ -364,17 +348,17 @@ public void childEvent( Thread.sleep(100); } - Map> replicationStatus = coordinator.getReplicationStatus(); + Map> replicationStatus = coordinator.getReplicationStatus(); Assert.assertNotNull(replicationStatus); Assert.assertEquals(1, replicationStatus.entrySet().size()); - CountingMap dataSourceMap = replicationStatus.get(tier); + Object2LongMap dataSourceMap = replicationStatus.get(tier); Assert.assertNotNull(dataSourceMap); Assert.assertEquals(1, dataSourceMap.size()); Assert.assertNotNull(dataSourceMap.get(dataSource)); // Simulated the adding of segment to druidServer during SegmentChangeRequestLoad event // The load rules asks for 2 replicas, therefore 1 replica should still be pending - Assert.assertEquals(1L, dataSourceMap.get(dataSource).get()); + Assert.assertEquals(1L, dataSourceMap.getLong(dataSource)); coordinator.stop(); leaderUnannouncerLatch.await(); diff --git a/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java index bb920a36198d..a6e2e5673d41 100644 --- a/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java +++ b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java @@ -41,6 +41,7 @@ public void loadSegment( segmentsToLoad.add(segment); } + @Override public ConcurrentSkipListSet getSegmentsToLoad() { return segmentsToLoad; diff --git a/server/src/test/java/io/druid/server/coordinator/ServerHolderTest.java b/server/src/test/java/io/druid/server/coordinator/ServerHolderTest.java new file mode 100644 index 000000000000..33e6417bd331 --- /dev/null +++ b/server/src/test/java/io/druid/server/coordinator/ServerHolderTest.java @@ -0,0 +1,246 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.coordinator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.druid.client.ImmutableDruidDataSource; +import io.druid.client.ImmutableDruidServer; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; + +public class ServerHolderTest +{ + private static final List segments = ImmutableList.of( + new DataSegment( + "test", + new Interval("2015-04-12/2015-04-13"), + "1", + ImmutableMap.of("containerName", "container1", "blobPath", "blobPath1"), + null, + null, + NoneShardSpec.instance(), + 0, + 1 + ), + new DataSegment( + "test", + new Interval("2015-04-12/2015-04-13"), + "1", + ImmutableMap.of("containerName", "container2", "blobPath", "blobPath2"), + null, + null, + NoneShardSpec.instance(), + 0, + 1 + ) + ); + + private static final Map dataSources = ImmutableMap.of( + "src1", + new ImmutableDruidDataSource( + "src1", + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableSet.of() + ), + "src2", + new ImmutableDruidDataSource( + "src2", + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableSet.of() + ) + ); + + @Test + public void testCompareTo() throws Exception + { + // available size of 100 + final ServerHolder h1 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + // available size of 100 + final ServerHolder h2 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 200L, ServerType.HISTORICAL, "tier1", 0), + 100L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + // available size of 10 + final ServerHolder h3 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 1000L, ServerType.HISTORICAL, "tier1", 0), + 990L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + // available size of 50 + final ServerHolder h4 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 50L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + Assert.assertEquals(0, h1.compareTo(h2)); + Assert.assertEquals(-1, h3.compareTo(h1)); + Assert.assertEquals(-1, h3.compareTo(h4)); + } + + @Test + public void testEquals() throws Exception + { + final ServerHolder h1 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + final ServerHolder h2 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name2", "host1", 200L, ServerType.HISTORICAL, "tier1", 0), + 100L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + final ServerHolder h3 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host2", 200L, ServerType.HISTORICAL, "tier1", 0), + 100L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + final ServerHolder h4 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 200L, ServerType.HISTORICAL, "tier2", 0), + 100L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + final ServerHolder h5 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", 100L, ServerType.REALTIME, "tier1", 0), + 0L, + ImmutableMap.of( + "src1", + dataSources.get("src1") + ), + ImmutableMap.of( + "segment1", + segments.get(0) + ) + ), + new LoadQueuePeonTester() + ); + + Assert.assertEquals(h1, h2); + Assert.assertNotEquals(h1, h3); + Assert.assertNotEquals(h1, h4); + Assert.assertNotEquals(h1, h5); + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java index 955d1de29969..9bd095be152b 100644 --- a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java +++ b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java @@ -38,7 +38,7 @@ import org.joda.time.Interval; import org.junit.Test; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class DruidCoordinatorCleanupOvershadowedTest @@ -63,6 +63,7 @@ public class DruidCoordinatorCleanupOvershadowedTest .interval(new Interval(start, start.plusHours(1))) .version("2") .build(); + @Test public void testRun() { @@ -70,14 +71,17 @@ public void testRun() availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); druidCluster = new DruidCluster( - ImmutableMap.of("normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create(Arrays.asList( - new ServerHolder(druidServer, mockPeon - ))))); + null, + ImmutableMap.of("normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( + Collections.singletonList(new ServerHolder(druidServer, mockPeon)) + ))); EasyMock.expect(druidServer.getDataSources()) .andReturn(ImmutableList.of(druidDataSource)) .anyTimes(); - EasyMock.expect(druidDataSource.getSegments()).andReturn(ImmutableSet.of(segmentV1, segmentV2)).anyTimes(); + EasyMock.expect(druidDataSource.getSegments()) + .andReturn(ImmutableSet.of(segmentV1, segmentV2)) + .anyTimes(); EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); coordinator.removeSegment(segmentV1); coordinator.removeSegment(segmentV0); diff --git a/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 1255a5df5e00..fe461117bd06 100644 --- a/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Ordering; import io.druid.client.DruidServer; +import io.druid.server.coordination.ServerType; import io.druid.server.coordinator.CoordinatorStats; import io.druid.server.coordinator.DruidCluster; import io.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -107,7 +108,7 @@ public void setUp() throws Exception "serverHot2", "hostHot2", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).addDataSegment(smallSegment.getIdentifier(), smallSegment) @@ -121,7 +122,7 @@ public void setUp() throws Exception "serverHot1", "hostHot1", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).addDataSegment(largeSegments.get(0).getIdentifier(), largeSegments.get(0)) @@ -135,7 +136,7 @@ public void setUp() throws Exception "serverNorm1", "hostNorm1", 1000, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ).addDataSegment(largeSegments.get(1).getIdentifier(), largeSegments.get(1)) @@ -149,7 +150,7 @@ public void setUp() throws Exception "serverNorm2", "hostNorm2", 100, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ).addDataSegment(largeSegments.get(2).getIdentifier(), largeSegments.get(2)) @@ -164,7 +165,7 @@ public void setUp() throws Exception "serverHot3", "hostHot3", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).addDataSegment(largeSegments2.get(0).getIdentifier(), largeSegments2.get(0)) @@ -178,7 +179,7 @@ public void setUp() throws Exception "serverNorm3", "hostNorm3", 100, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ).addDataSegment(largeSegments2.get(1).getIdentifier(), largeSegments2.get(1)) @@ -188,6 +189,7 @@ public void setUp() throws Exception ); druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -231,8 +233,8 @@ public void testBroadcastToSingleDataSource() smallSegment ); - assertEquals(3, stats.getGlobalStats().get(LoadRule.ASSIGNED_COUNT).intValue()); - assertTrue(stats.getPerTierStats().isEmpty()); + assertEquals(3L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + assertEquals(false, stats.hasPerTierStats()); assertTrue( holdersOfLargeSegments.stream() @@ -271,8 +273,8 @@ public void testBroadcastToMultipleDataSources() smallSegment ); - assertEquals(5, stats.getGlobalStats().get(LoadRule.ASSIGNED_COUNT).intValue()); - assertTrue(stats.getPerTierStats().isEmpty()); + assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + assertEquals(false, stats.hasPerTierStats()); assertTrue( holdersOfLargeSegments.stream() @@ -309,8 +311,8 @@ public void testBroadcastToAllServers() smallSegment ); - assertEquals(6, stats.getGlobalStats().get(LoadRule.ASSIGNED_COUNT).intValue()); - assertTrue(stats.getPerTierStats().isEmpty()); + assertEquals(6L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + assertEquals(false, stats.hasPerTierStats()); assertTrue( druidCluster.getAllServers().stream() diff --git a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java index 33597215f81f..64d91e7c1248 100644 --- a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java @@ -34,6 +34,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.DruidServer; import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.coordination.ServerType; import io.druid.server.coordinator.BalancerStrategy; import io.druid.server.coordinator.CoordinatorStats; import io.druid.server.coordinator.CostBalancerStrategyFactory; @@ -158,6 +159,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) }; DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -167,7 +169,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -183,7 +185,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ).toImmutableDruidServer(), @@ -211,8 +213,8 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) segment ); - Assert.assertTrue(stats.getPerTierStats().get(LoadRule.ASSIGNED_COUNT).get("hot").get() == 1); - Assert.assertTrue(stats.getPerTierStats().get(LoadRule.ASSIGNED_COUNT).get(DruidServer.DEFAULT_TIER).get() == 2); + Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); + Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER)); exec.shutdown(); } @@ -267,7 +269,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ); @@ -276,12 +278,13 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverNorm", "hostNorm", 1000, - "historical", + ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0 ); server2.addDataSegment(segment.getIdentifier(), segment); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -321,8 +324,8 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) segment ); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("hot").get() == 1); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get(DruidServer.DEFAULT_TIER).get() == 1); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", DruidServer.DEFAULT_TIER)); exec.shutdown(); } @@ -374,6 +377,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) }; DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -383,7 +387,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ).toImmutableDruidServer(), @@ -411,7 +415,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) segment ); - Assert.assertTrue(stats.getPerTierStats().get(LoadRule.ASSIGNED_COUNT).get("hot").get() == 1); + Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); exec.shutdown(); } @@ -466,7 +470,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverHot", "hostHot", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ); @@ -474,7 +478,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) "serverHo2t", "hostHot2", 1000, - "historical", + ServerType.HISTORICAL, "hot", 0 ); @@ -482,6 +486,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) server2.addDataSegment(segment.getIdentifier(), segment); DruidCluster druidCluster = new DruidCluster( + null, ImmutableMap.of( "hot", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create( @@ -516,7 +521,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp) segment ); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("hot").get() == 1); + Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); exec.shutdown(); } } diff --git a/server/src/test/java/io/druid/server/http/CoordinatorRedirectInfoTest.java b/server/src/test/java/io/druid/server/http/CoordinatorRedirectInfoTest.java index 1bec282bcdc7..56feb114a901 100644 --- a/server/src/test/java/io/druid/server/http/CoordinatorRedirectInfoTest.java +++ b/server/src/test/java/io/druid/server/http/CoordinatorRedirectInfoTest.java @@ -40,11 +40,26 @@ public void setUp() } @Test - public void testDoLocal() + public void testDoLocalWhenLeading() { EasyMock.expect(druidCoordinator.isLeader()).andReturn(true).anyTimes(); EasyMock.replay(druidCoordinator); Assert.assertTrue(coordinatorRedirectInfo.doLocal(null)); + Assert.assertTrue(coordinatorRedirectInfo.doLocal("/druid/coordinator/v1/leader")); + Assert.assertTrue(coordinatorRedirectInfo.doLocal("/druid/coordinator/v1/isLeader")); + Assert.assertTrue(coordinatorRedirectInfo.doLocal("/druid/coordinator/v1/other/path")); + EasyMock.verify(druidCoordinator); + } + + @Test + public void testDoLocalWhenNotLeading() + { + EasyMock.expect(druidCoordinator.isLeader()).andReturn(false).anyTimes(); + EasyMock.replay(druidCoordinator); + Assert.assertFalse(coordinatorRedirectInfo.doLocal(null)); + Assert.assertTrue(coordinatorRedirectInfo.doLocal("/druid/coordinator/v1/leader")); + Assert.assertTrue(coordinatorRedirectInfo.doLocal("/druid/coordinator/v1/isLeader")); + Assert.assertFalse(coordinatorRedirectInfo.doLocal("/druid/coordinator/v1/other/path")); EasyMock.verify(druidCoordinator); } diff --git a/server/src/test/java/io/druid/server/http/CoordinatorResourceTest.java b/server/src/test/java/io/druid/server/http/CoordinatorResourceTest.java new file mode 100644 index 000000000000..7fe0637a326d --- /dev/null +++ b/server/src/test/java/io/druid/server/http/CoordinatorResourceTest.java @@ -0,0 +1,76 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.http; + +import com.google.common.collect.ImmutableMap; +import io.druid.server.coordinator.DruidCoordinator; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.ws.rs.core.Response; + +public class CoordinatorResourceTest +{ + private DruidCoordinator mock; + + @Before + public void setUp() + { + mock = EasyMock.createStrictMock(DruidCoordinator.class); + } + + @After + public void tearDown() + { + EasyMock.verify(mock); + } + + @Test + public void testLeader() + { + EasyMock.expect(mock.getCurrentLeader()).andReturn("boz").once(); + EasyMock.replay(mock); + + final Response response = new CoordinatorResource(mock).getLeader(); + Assert.assertEquals("boz", response.getEntity()); + Assert.assertEquals(200, response.getStatus()); + } + + @Test + public void testIsLeader() + { + EasyMock.expect(mock.isLeader()).andReturn(true).once(); + EasyMock.expect(mock.isLeader()).andReturn(false).once(); + EasyMock.replay(mock); + + // true + final Response response1 = new CoordinatorResource(mock).isLeader(); + Assert.assertEquals(ImmutableMap.of("leader", true), response1.getEntity()); + Assert.assertEquals(200, response1.getStatus()); + + // false + final Response response2 = new CoordinatorResource(mock).isLeader(); + Assert.assertEquals(ImmutableMap.of("leader", false), response2.getEntity()); + Assert.assertEquals(404, response2.getStatus()); + } +} diff --git a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java index 71147cdaa7bb..6404ec2f38e8 100644 --- a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java +++ b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java @@ -25,6 +25,7 @@ import io.druid.client.DruidDataSource; import io.druid.client.DruidServer; import io.druid.client.indexing.IndexingServiceClient; +import io.druid.server.coordination.ServerType; import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; @@ -328,7 +329,7 @@ public void testSimpleGetTheDataSourceManyTiers() throws Exception @Test public void testGetSegmentDataSourceIntervals() { - server = new DruidServer("who", "host", 1234, "historical", "tier1", 0); + server = new DruidServer("who", "host", 1234, ServerType.HISTORICAL, "tier1", 0); server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0)); server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1)); server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2)); @@ -378,7 +379,7 @@ public void testGetSegmentDataSourceIntervals() @Test public void testGetSegmentDataSourceSpecificInterval() { - server = new DruidServer("who", "host", 1234, "historical", "tier1", 0); + server = new DruidServer("who", "host", 1234, ServerType.HISTORICAL, "tier1", 0); server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0)); server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1)); server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2)); diff --git a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java index 4fb50795c85a..d3cb3aa157e9 100644 --- a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java +++ b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import io.druid.client.DruidServer; import io.druid.client.InventoryView; +import io.druid.server.coordination.ServerType; import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; @@ -92,7 +93,7 @@ public void setUp() 5 ) ); - server = new DruidServer("who", "host", 1234, "historical", "tier1", 0); + server = new DruidServer("who", "host", 1234, ServerType.HISTORICAL, "tier1", 0); server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0)); server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1)); server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2)); diff --git a/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java b/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java index 394c273b17a5..23e0b2ce91f2 100644 --- a/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java +++ b/server/src/test/java/io/druid/server/http/LookupCoordinatorResourceTest.java @@ -23,9 +23,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteSource; +import com.google.common.net.HostAndPort; import io.druid.audit.AuditInfo; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.StringUtils; +import io.druid.query.lookup.LookupsState; import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.server.lookup.cache.LookupExtractorFactoryMapContainer; import org.easymock.Capture; @@ -77,6 +79,16 @@ public InputStream openStream() throws IOException } }; + private static final HostAndPort LOOKUP_NODE = HostAndPort.fromParts("localhost", 1111); + + private static final LookupsState LOOKUP_STATE = new LookupsState( + ImmutableMap.of(LOOKUP_NAME, SINGLE_LOOKUP), null, null + ); + + private static final Map> NODES_LOOKUP_STATE = ImmutableMap.of( + LOOKUP_NODE, LOOKUP_STATE + ); + @Test public void testSimpleGet() { @@ -830,4 +842,225 @@ public void testExceptionalGetTier() Assert.assertEquals(ImmutableMap.of("error", errMsg), response.getEntity()); EasyMock.verify(lookupCoordinatorManager); } + + @Test + public void testGetAllLookupsStatus() throws Exception + { + final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( + LookupCoordinatorManager.class + ); + EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(SINGLE_TIER_MAP); + EasyMock.expect(lookupCoordinatorManager.getLastKnownLookupsStateOnNodes()).andReturn(NODES_LOOKUP_STATE); + EasyMock.expect(lookupCoordinatorManager.discoverNodesInTier(LOOKUP_TIER)).andReturn(ImmutableList.of(LOOKUP_NODE)); + + EasyMock.replay(lookupCoordinatorManager); + + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + lookupCoordinatorManager, + mapper, + mapper + ); + + final Response response = lookupCoordinatorResource.getAllLookupsStatus(false); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of( + LOOKUP_TIER, + ImmutableMap.of( + LOOKUP_NAME, + new LookupCoordinatorResource.LookupStatus(true, null) + ) + ), response.getEntity() + ); + + EasyMock.verify(lookupCoordinatorManager); + } + + @Test + public void testGetLookupStatusForTier() throws Exception + { + final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( + LookupCoordinatorManager.class + ); + EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(SINGLE_TIER_MAP); + EasyMock.expect(lookupCoordinatorManager.discoverNodesInTier(LOOKUP_TIER)).andReturn(ImmutableList.of(LOOKUP_NODE)); + EasyMock.expect(lookupCoordinatorManager.getLastKnownLookupsStateOnNodes()).andReturn(NODES_LOOKUP_STATE); + + EasyMock.replay(lookupCoordinatorManager); + + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + lookupCoordinatorManager, + mapper, + mapper + ); + + final Response response = lookupCoordinatorResource.getLookupStatusForTier(LOOKUP_TIER, false); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of( + LOOKUP_NAME, + new LookupCoordinatorResource.LookupStatus(true, null) + ), response.getEntity() + ); + + EasyMock.verify(lookupCoordinatorManager); + } + + @Test + public void testGetSpecificLookupStatus() throws Exception + { + final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( + LookupCoordinatorManager.class + ); + EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(SINGLE_TIER_MAP); + EasyMock.expect(lookupCoordinatorManager.discoverNodesInTier(LOOKUP_TIER)).andReturn(ImmutableList.of(LOOKUP_NODE)); + EasyMock.expect(lookupCoordinatorManager.getLastKnownLookupsStateOnNodes()).andReturn(NODES_LOOKUP_STATE); + + EasyMock.replay(lookupCoordinatorManager); + + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + lookupCoordinatorManager, + mapper, + mapper + ); + + final Response response = lookupCoordinatorResource.getSpecificLookupStatus(LOOKUP_TIER, LOOKUP_NAME, false); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + new LookupCoordinatorResource.LookupStatus(true, null), response.getEntity() + ); + + EasyMock.verify(lookupCoordinatorManager); + } + + @Test + public void testGetLookupStatusDetailedTrue() + { + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + EasyMock.createStrictMock(LookupCoordinatorManager.class), + mapper, + mapper + ); + + HostAndPort newNode = HostAndPort.fromParts("localhost", 4352); + Assert.assertEquals( + new LookupCoordinatorResource.LookupStatus(false, ImmutableList.of(newNode)), + lookupCoordinatorResource.getLookupStatus( + LOOKUP_NAME, + SINGLE_LOOKUP, + ImmutableList.of(LOOKUP_NODE, newNode), + NODES_LOOKUP_STATE, + true + ) + ); + } + + @Test + public void testGetLookupStatusDetailedFalse() + { + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + EasyMock.createStrictMock(LookupCoordinatorManager.class), + mapper, + mapper + ); + + HostAndPort newNode = HostAndPort.fromParts("localhost", 4352); + Assert.assertEquals( + new LookupCoordinatorResource.LookupStatus(false, null), + lookupCoordinatorResource.getLookupStatus( + LOOKUP_NAME, + SINGLE_LOOKUP, + ImmutableList.of(LOOKUP_NODE, newNode), + NODES_LOOKUP_STATE, + false + ) + ); + } + + @Test + public void testGetAllNodesStatus() throws Exception + { + final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( + LookupCoordinatorManager.class + ); + EasyMock.expect(lookupCoordinatorManager.getKnownLookups()).andReturn(SINGLE_TIER_MAP); + EasyMock.expect(lookupCoordinatorManager.getLastKnownLookupsStateOnNodes()).andReturn(NODES_LOOKUP_STATE); + EasyMock.expect(lookupCoordinatorManager.discoverNodesInTier(LOOKUP_TIER)).andReturn(ImmutableList.of(LOOKUP_NODE)); + + EasyMock.replay(lookupCoordinatorManager); + + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + lookupCoordinatorManager, + mapper, + mapper + ); + + final Response response = lookupCoordinatorResource.getAllNodesStatus(false); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of( + LOOKUP_TIER, + ImmutableMap.of( + LOOKUP_NODE, + LOOKUP_STATE + ) + ), response.getEntity() + ); + + EasyMock.verify(lookupCoordinatorManager); + } + + @Test + public void testGetNodesStatusInTier() throws Exception + { + final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( + LookupCoordinatorManager.class + ); + EasyMock.expect(lookupCoordinatorManager.getLastKnownLookupsStateOnNodes()).andReturn(NODES_LOOKUP_STATE); + EasyMock.expect(lookupCoordinatorManager.discoverNodesInTier(LOOKUP_TIER)).andReturn(ImmutableList.of(LOOKUP_NODE)); + + EasyMock.replay(lookupCoordinatorManager); + + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + lookupCoordinatorManager, + mapper, + mapper + ); + + final Response response = lookupCoordinatorResource.getNodesStatusInTier(LOOKUP_TIER); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of( + LOOKUP_NODE, + LOOKUP_STATE + ), response.getEntity() + ); + + EasyMock.verify(lookupCoordinatorManager); + } + + @Test + public void testGetSpecificNodeStatus() throws Exception + { + final LookupCoordinatorManager lookupCoordinatorManager = EasyMock.createStrictMock( + LookupCoordinatorManager.class + ); + EasyMock.expect(lookupCoordinatorManager.getLastKnownLookupsStateOnNodes()).andReturn(NODES_LOOKUP_STATE); + + EasyMock.replay(lookupCoordinatorManager); + + final LookupCoordinatorResource lookupCoordinatorResource = new LookupCoordinatorResource( + lookupCoordinatorManager, + mapper, + mapper + ); + + final Response response = lookupCoordinatorResource.getSpecificNodeStatus(LOOKUP_TIER, LOOKUP_NODE); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + LOOKUP_STATE, response.getEntity() + ); + + EasyMock.verify(lookupCoordinatorManager); + } } diff --git a/server/src/test/java/io/druid/server/http/ServersResourceTest.java b/server/src/test/java/io/druid/server/http/ServersResourceTest.java index 0695bf6750bf..80e4b97ffb05 100644 --- a/server/src/test/java/io/druid/server/http/ServersResourceTest.java +++ b/server/src/test/java/io/druid/server/http/ServersResourceTest.java @@ -24,6 +24,8 @@ import io.druid.client.CoordinatorServerView; import io.druid.client.DruidServer; import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -41,7 +43,7 @@ public class ServersResourceTest { @Before public void setUp() { - DruidServer dummyServer = new DruidServer("dummy", "host", 1234L, "type", "tier", 0); + DruidServer dummyServer = new DruidServer("dummy", "host", 1234L, ServerType.HISTORICAL, "tier", 0); DataSegment segment = DataSegment.builder() .dataSource("dataSource") .interval(new Interval("2016-03-22T14Z/2016-03-22T15Z")) @@ -65,7 +67,7 @@ public void testGetClusterServersFull() throws Exception String result = objectMapper.writeValueAsString(res.getEntity()); String expected = "[{\"host\":\"host\"," + "\"maxSize\":1234," - + "\"type\":\"type\"," + + "\"type\":\"historical\"," + "\"tier\":\"tier\"," + "\"priority\":0," + "\"segments\":{\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\":" @@ -80,7 +82,7 @@ public void testGetClusterServersSimple() throws Exception { Response res = serversResource.getClusterServers(null, "simple"); String result = objectMapper.writeValueAsString(res.getEntity()); - String expected = "[{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"type\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}]"; + String expected = "[{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"historical\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}]"; Assert.assertEquals(expected, result); } @@ -91,7 +93,7 @@ public void testGetServerFull() throws Exception String result = objectMapper.writeValueAsString(res.getEntity()); String expected = "{\"host\":\"host\"," + "\"maxSize\":1234," - + "\"type\":\"type\"," + + "\"type\":\"historical\"," + "\"tier\":\"tier\"," + "\"priority\":0," + "\"segments\":{\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\":" @@ -106,8 +108,29 @@ public void testGetServerSimple() throws Exception { Response res = serversResource.getServer(server.getName(), "simple"); String result = objectMapper.writeValueAsString(res.getEntity()); - String expected = "{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"type\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}"; + String expected = "{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"historical\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}"; Assert.assertEquals(expected, result); } + @Test + public void testDruidServerSerde() throws Exception + { + DruidServer server = new DruidServer("dummy", "dummyHost", 1234, ServerType.HISTORICAL, "dummyTier", 1); + String serverJson = objectMapper.writeValueAsString(server); + String expected = "{\"name\":\"dummy\",\"host\":\"dummyHost\",\"maxSize\":1234,\"type\":\"historical\",\"tier\":\"dummyTier\",\"priority\":1}"; + Assert.assertEquals(expected, serverJson); + DruidServer deserializedServer = objectMapper.readValue(serverJson, DruidServer.class); + Assert.assertEquals(server, deserializedServer); + } + + @Test + public void testDruidServerMetadataSerde() throws Exception + { + DruidServerMetadata metadata = new DruidServerMetadata("dummy", "host", 1234, ServerType.HISTORICAL, "tier", 1); + String metadataJson = objectMapper.writeValueAsString(metadata); + String expected = "{\"name\":\"dummy\",\"host\":\"host\",\"maxSize\":1234,\"type\":\"historical\",\"tier\":\"tier\",\"priority\":1}"; + Assert.assertEquals(expected, metadataJson); + DruidServerMetadata deserializedMetadata = objectMapper.readValue(metadataJson, DruidServerMetadata.class); + Assert.assertEquals(metadata, deserializedMetadata); + } } diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index c7c0247bd14a..1d1e8f93c8e9 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -1117,11 +1117,13 @@ public void testLookupManagementLoop() throws Exception LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig = new LookupCoordinatorManagerConfig() { + @Override public long getInitialDelay() { return 1; } + @Override public int getThreadPoolSize() { return 2; @@ -1135,7 +1137,7 @@ public int getThreadPoolSize() lookupsCommunicator ); - Assert.assertNull(manager.knownOldState.get()); + Assert.assertTrue(manager.knownOldState.get().isEmpty()); manager.start(); @@ -1335,6 +1337,7 @@ public void testLookupDiscoverAll() throws Exception configManager, lookupCoordinatorManagerConfig ); + manager.start(); Assert.assertEquals(fakeChildren, manager.discoverTiers()); EasyMock.verify(discoverer); } @@ -1371,12 +1374,10 @@ public void describeTo(Description description) configManager, lookupCoordinatorManagerConfig ); - try { - manager.discoverTiers(); - } - finally { - EasyMock.verify(discoverer); - } + + manager.start(); + manager.discoverTiers(); + EasyMock.verify(discoverer); } //tests that lookups stored in db from 0.10.0 are converted and restored. diff --git a/server/src/test/java/io/druid/server/metrics/HistoricalMetricsMonitorTest.java b/server/src/test/java/io/druid/server/metrics/HistoricalMetricsMonitorTest.java index 68558cc1db9f..0acc6bd6f088 100644 --- a/server/src/test/java/io/druid/server/metrics/HistoricalMetricsMonitorTest.java +++ b/server/src/test/java/io/druid/server/metrics/HistoricalMetricsMonitorTest.java @@ -27,7 +27,7 @@ import com.metamx.emitter.service.ServiceEventBuilder; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.DruidServerConfig; -import io.druid.server.coordination.ServerManager; +import io.druid.server.SegmentManager; import io.druid.server.coordination.ZkCoordinator; import io.druid.timeline.DataSegment; import org.easymock.Capture; @@ -47,7 +47,7 @@ public class HistoricalMetricsMonitorTest extends EasyMockSupport { private DruidServerConfig druidServerConfig; - private ServerManager serverManager; + private SegmentManager segmentManager; private ZkCoordinator zkCoordinator; private ServiceEmitter serviceEmitter; @@ -55,7 +55,7 @@ public class HistoricalMetricsMonitorTest extends EasyMockSupport public void setUp() { druidServerConfig = EasyMock.createStrictMock(DruidServerConfig.class); - serverManager = EasyMock.createStrictMock(ServerManager.class); + segmentManager = EasyMock.createStrictMock(SegmentManager.class); zkCoordinator = EasyMock.createStrictMock(ZkCoordinator.class); serviceEmitter = EasyMock.createStrictMock(ServiceEmitter.class); } @@ -84,17 +84,17 @@ public void testSimple() EasyMock.expect(zkCoordinator.getPendingDeleteSnapshot()).andReturn(ImmutableList.of(dataSegment)).once(); EasyMock.expect(druidServerConfig.getTier()).andReturn(tier).once(); EasyMock.expect(druidServerConfig.getPriority()).andReturn(priority).once(); - EasyMock.expect(serverManager.getDataSourceSizes()).andReturn(ImmutableMap.of(dataSource, size)); + EasyMock.expect(segmentManager.getDataSourceSizes()).andReturn(ImmutableMap.of(dataSource, size)); EasyMock.expect(druidServerConfig.getTier()).andReturn(tier).once(); EasyMock.expect(druidServerConfig.getPriority()).andReturn(priority).once(); EasyMock.expect(druidServerConfig.getMaxSize()).andReturn(maxSize).times(2); - EasyMock.expect(serverManager.getDataSourceCounts()).andReturn(ImmutableMap.of(dataSource, 1L)); + EasyMock.expect(segmentManager.getDataSourceCounts()).andReturn(ImmutableMap.of(dataSource, 1L)); EasyMock.expect(druidServerConfig.getTier()).andReturn(tier).once(); EasyMock.expect(druidServerConfig.getPriority()).andReturn(priority).once(); final HistoricalMetricsMonitor monitor = new HistoricalMetricsMonitor( druidServerConfig, - serverManager, + segmentManager, zkCoordinator ); @@ -102,9 +102,9 @@ public void testSimple() serviceEmitter.emit(EasyMock.capture(eventCapture)); EasyMock.expectLastCall().times(5); - EasyMock.replay(druidServerConfig, serverManager, zkCoordinator, serviceEmitter); + EasyMock.replay(druidServerConfig, segmentManager, zkCoordinator, serviceEmitter); monitor.doMonitor(serviceEmitter); - EasyMock.verify(druidServerConfig, serverManager, zkCoordinator, serviceEmitter); + EasyMock.verify(druidServerConfig, segmentManager, zkCoordinator, serviceEmitter); final String host = "host"; final String service = "service"; @@ -119,7 +119,7 @@ public Map apply( @Nullable ServiceEventBuilder input ) { - final HashMap map = new HashMap<>(input.build(host, service).toMap()); + final HashMap map = new HashMap<>(input.build(service, host).toMap()); Assert.assertNotNull(map.remove("feed")); Assert.assertNotNull(map.remove("timestamp")); Assert.assertNotNull(map.remove("service")); diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 6446e2ec208f..89d92f175f1a 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -23,7 +23,6 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.name.Names; - import io.airlift.airline.Command; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.CacheMonitor; @@ -37,13 +36,16 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.lookup.LookupModule; -import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.QueryResource; +import io.druid.server.SegmentManager; import io.druid.server.coordination.ServerManager; +import io.druid.server.coordination.ServerType; import io.druid.server.coordination.ZkCoordinator; import io.druid.server.http.HistoricalResource; +import io.druid.server.http.SegmentListerResource; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.metrics.MetricsModule; +import io.druid.server.metrics.QueryCountStatsProvider; import org.eclipse.jetty.server.Server; import java.util.List; @@ -78,14 +80,16 @@ public void configure(Binder binder) // register Server before binding ZkCoordinator to ensure HTTP endpoints are available immediately LifecycleModule.register(binder, Server.class); binder.bind(ServerManager.class).in(LazySingleton.class); + binder.bind(SegmentManager.class).in(LazySingleton.class); binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); binder.bind(QuerySegmentWalker.class).to(ServerManager.class).in(LazySingleton.class); - binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("historical")); + binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.HISTORICAL)); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class); Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, HistoricalResource.class); + Jerseys.addResource(binder, SegmentListerResource.class); LifecycleModule.register(binder, QueryResource.class); LifecycleModule.register(binder, ZkCoordinator.class); diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 530e224ae539..d236ac593e75 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -84,6 +84,8 @@ import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig; import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.server.coordination.ServerType; +import io.druid.server.http.SegmentListerResource; import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.QueryResource; import io.druid.server.initialization.jetty.ChatHandlerServerModule; @@ -204,8 +206,9 @@ public void configure(Binder binder) binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class); Jerseys.addResource(binder, QueryResource.class); + Jerseys.addResource(binder, SegmentListerResource.class); LifecycleModule.register(binder, QueryResource.class); - binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(nodeType)); + binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.fromString(nodeType))); LifecycleModule.register(binder, Server.class); } diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 9d9122d8a435..b6516fb4c66d 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -20,11 +20,11 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Module; import com.google.inject.name.Names; - import io.airlift.airline.Command; import io.druid.client.DruidServer; import io.druid.client.InventoryView; @@ -40,7 +40,9 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.concurrent.Executor; @@ -118,6 +120,18 @@ public Iterable getInventory() { return ImmutableList.of(); } + + @Override + public boolean isStarted() + { + return true; + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + return false; + } } private static class NoopDataSegmentPusher implements DataSegmentPusher @@ -141,6 +155,12 @@ public DataSegment push(File file, DataSegment segment) throws IOException { return segment; } + + @Override + public Map makeLoadSpec(URI uri) + { + return ImmutableMap.of(); + } } private static class NoopDataSegmentAnnouncer implements DataSegmentAnnouncer @@ -168,11 +188,5 @@ public void unannounceSegments(Iterable segments) throws IOExceptio { // do nothing } - - @Override - public boolean isAnnounced(DataSegment segment) - { - return false; - } } } diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index fa14574cfca8..8901a4401909 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -51,6 +51,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.DruidProcessingConfig; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -471,7 +472,7 @@ private static Sequence executeQuery(final Injector injector, final Query final QueryRunner runner = factory.createRunner(new QueryableIndexSegment("segment", index)); final Sequence results = factory.getToolchest().mergeResults( factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(runner)) - ).run(query, Maps.newHashMap()); + ).run(QueryPlus.wrap(query), Maps.newHashMap()); return (Sequence) results; } diff --git a/services/src/main/java/io/druid/cli/GuiceRunnable.java b/services/src/main/java/io/druid/cli/GuiceRunnable.java index f07edaeb8f61..82d8155611cd 100644 --- a/services/src/main/java/io/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/io/druid/cli/GuiceRunnable.java @@ -77,9 +77,10 @@ public Lifecycle initLifecycle(Injector injector) final StartupLoggingConfig startupLoggingConfig = injector.getInstance(StartupLoggingConfig.class); log.info( - "Starting up with processors[%,d], memory[%,d].", + "Starting up with processors[%,d], memory[%,d], maxMemory[%,d].", Runtime.getRuntime().availableProcessors(), - Runtime.getRuntime().totalMemory() + Runtime.getRuntime().totalMemory(), + Runtime.getRuntime().maxMemory() ); if (startupLoggingConfig.isLogProperties()) { diff --git a/services/src/main/java/io/druid/cli/InsertSegment.java b/services/src/main/java/io/druid/cli/InsertSegment.java index 2d43b629c470..f6b1850d2b6f 100644 --- a/services/src/main/java/io/druid/cli/InsertSegment.java +++ b/services/src/main/java/io/druid/cli/InsertSegment.java @@ -92,7 +92,7 @@ public void run() indexerMetadataStorageCoordinator = injector.getInstance(IndexerMetadataStorageCoordinator.class); final DataSegmentFinder dataSegmentFinder = injector.getInstance(DataSegmentFinder.class); - log.info("Start seraching segments under [%s]", workingDirPath); + log.info("Start searching segments under [%s]", workingDirPath); Set segments = null; try { diff --git a/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java b/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java index 9267f8a18734..5e8168987897 100644 --- a/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java +++ b/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java @@ -26,10 +26,10 @@ public class Log4JShutdownPropertyChecker implements PropertyChecker @Override public void checkProperties(Properties properties) { - if (!properties.contains("log4j.shutdownCallbackRegistry")) { + if (!properties.containsKey("log4j.shutdownCallbackRegistry")) { properties.setProperty("log4j.shutdownCallbackRegistry", "io.druid.common.config.Log4jShutdown"); } - if (!properties.contains("log4j.shutdownHookEnabled")) { + if (!properties.containsKey("log4j.shutdownHookEnabled")) { properties.setProperty("log4j.shutdownHookEnabled", "true"); } } diff --git a/services/src/main/java/io/druid/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index aa0e357fc493..e63838587ffa 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -39,9 +39,13 @@ import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig; import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; -import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.QueryResource; +import io.druid.server.SegmentManager; +import io.druid.server.coordination.ServerType; +import io.druid.server.coordination.ZkCoordinator; +import io.druid.server.http.SegmentListerResource; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.metrics.QueryCountStatsProvider; import org.eclipse.jetty.server.Server; import java.util.List; @@ -101,11 +105,16 @@ public void configure(Binder binder) binder.install(new CacheModule()); binder.bind(QuerySegmentWalker.class).to(RealtimeManager.class).in(ManageLifecycle.class); - binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("realtime")); + binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.REALTIME)); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class); Jerseys.addResource(binder, QueryResource.class); + Jerseys.addResource(binder, SegmentListerResource.class); LifecycleModule.register(binder, QueryResource.class); LifecycleModule.register(binder, Server.class); + + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + LifecycleModule.register(binder, ZkCoordinator.class); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java index 9927412e63a7..ac4f84b6423c 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java @@ -42,6 +42,7 @@ public class PlannerContext private final PlannerConfig plannerConfig; private final DateTime localNow; + private final long queryStartTimeMillis; private final Map queryContext; private PlannerContext( @@ -53,6 +54,7 @@ private PlannerContext( this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig"); this.queryContext = queryContext != null ? ImmutableMap.copyOf(queryContext) : ImmutableMap.of(); this.localNow = Preconditions.checkNotNull(localNow, "localNow"); + this.queryStartTimeMillis = System.currentTimeMillis(); } public static PlannerContext create( @@ -106,6 +108,11 @@ public Map getQueryContext() return queryContext; } + public long getQueryStartTimeMillis() + { + return queryStartTimeMillis; + } + public DataContext createDataContext(final JavaTypeFactory typeFactory) { class DruidDataContext implements DataContext diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java index 85dba6686a0a..e51c3ca43b20 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java @@ -21,6 +21,7 @@ import com.google.inject.Inject; import io.druid.query.QuerySegmentWalker; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.schema.DruidSchema; import org.apache.calcite.avatica.util.Casing; @@ -44,25 +45,28 @@ public class PlannerFactory private final QuerySegmentWalker walker; private final DruidOperatorTable operatorTable; private final PlannerConfig plannerConfig; + private final ServerConfig serverConfig; @Inject public PlannerFactory( final SchemaPlus rootSchema, final QuerySegmentWalker walker, final DruidOperatorTable operatorTable, - final PlannerConfig plannerConfig + final PlannerConfig plannerConfig, + final ServerConfig serverConfig ) { this.rootSchema = rootSchema; this.walker = walker; this.operatorTable = operatorTable; this.plannerConfig = plannerConfig; + this.serverConfig = serverConfig; } public DruidPlanner createPlanner(final Map queryContext) { final PlannerContext plannerContext = PlannerContext.create(plannerConfig, queryContext); - final QueryMaker queryMaker = new QueryMaker(walker, plannerContext); + final QueryMaker queryMaker = new QueryMaker(walker, plannerContext, serverConfig); final FrameworkConfig frameworkConfig = Frameworks .newConfigBuilder() .parserConfig( diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 5cbf91d4b182..5be2b554cea4 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -23,15 +23,16 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; import com.google.common.primitives.Doubles; import com.google.common.primitives.Ints; +import io.druid.client.DirectDruidClient; import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.DataSource; import io.druid.query.QueryDataSource; +import io.druid.query.QueryPlus; import io.druid.query.QuerySegmentWalker; import io.druid.query.Result; import io.druid.query.dimension.DimensionSpec; @@ -46,6 +47,7 @@ import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNResultValue; import io.druid.segment.column.Column; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerContext; import io.druid.sql.calcite.table.RowSignature; @@ -68,14 +70,17 @@ public class QueryMaker { private final QuerySegmentWalker walker; private final PlannerContext plannerContext; + private final ServerConfig serverConfig; public QueryMaker( final QuerySegmentWalker walker, - final PlannerContext plannerContext + final PlannerContext plannerContext, + final ServerConfig serverConfig ) { this.walker = walker; this.plannerContext = plannerContext; + this.serverConfig = serverConfig; } public PlannerContext getPlannerContext() @@ -178,12 +183,15 @@ public boolean hasNext() @Override public Sequence next() { - final SelectQuery queryWithPagination = baseQuery.withPagingSpec( - new PagingSpec( - pagingIdentifiers.get(), - plannerContext.getPlannerConfig().getSelectThreshold(), - true - ) + final SelectQuery queryWithPagination = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes( + baseQuery.withPagingSpec( + new PagingSpec( + pagingIdentifiers.get(), + plannerContext.getPlannerConfig().getSelectThreshold(), + true + ) + ), + serverConfig ); Hook.QUERY_PLAN.run(queryWithPagination); @@ -193,7 +201,13 @@ public Sequence next() return Sequences.concat( Sequences.map( - queryWithPagination.run(walker, Maps.newHashMap()), + QueryPlus.wrap(queryWithPagination) + .run(walker, + DirectDruidClient.makeResponseContextForQuery( + queryWithPagination, + plannerContext.getQueryStartTimeMillis() + ) + ), new Function, Sequence>() { @Override @@ -254,9 +268,14 @@ public void remove() private Sequence executeTimeseries( final DruidQueryBuilder queryBuilder, - final TimeseriesQuery query + final TimeseriesQuery baseQuery ) { + final TimeseriesQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes( + baseQuery, + serverConfig + ); + final List fieldList = queryBuilder.getRowType().getFieldList(); final List dimensions = queryBuilder.getGrouping().getDimensions(); final String timeOutputName = dimensions.isEmpty() ? null : Iterables.getOnlyElement(dimensions).getOutputName(); @@ -264,7 +283,11 @@ private Sequence executeTimeseries( Hook.QUERY_PLAN.run(query); return Sequences.map( - query.run(walker, Maps.newHashMap()), + QueryPlus.wrap(query) + .run( + walker, + DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis()) + ), new Function, Object[]>() { @Override @@ -290,16 +313,25 @@ public Object[] apply(final Result result) private Sequence executeTopN( final DruidQueryBuilder queryBuilder, - final TopNQuery query + final TopNQuery baseQuery ) { + final TopNQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes( + baseQuery, + serverConfig + ); + final List fieldList = queryBuilder.getRowType().getFieldList(); Hook.QUERY_PLAN.run(query); return Sequences.concat( Sequences.map( - query.run(walker, Maps.newHashMap()), + QueryPlus.wrap(query) + .run( + walker, + DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis()) + ), new Function, Sequence>() { @Override @@ -327,15 +359,23 @@ public Sequence apply(final Result result) private Sequence executeGroupBy( final DruidQueryBuilder queryBuilder, - final GroupByQuery query + final GroupByQuery baseQuery ) { + final GroupByQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes( + baseQuery, + serverConfig + ); + final List fieldList = queryBuilder.getRowType().getFieldList(); Hook.QUERY_PLAN.run(query); - return Sequences.map( - query.run(walker, Maps.newHashMap()), + QueryPlus.wrap(query) + .run( + walker, + DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis()) + ), new Function() { @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java b/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java index 1febd2ea343d..5dc6a1f272dc 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java @@ -54,6 +54,7 @@ public class DruidSemiJoinRule extends RelOptRule private static final Predicate IS_LEFT_OR_INNER = new Predicate() { + @Override public boolean apply(Join join) { final JoinRelType joinType = join.getJoinType(); @@ -64,6 +65,7 @@ public boolean apply(Join join) private static final Predicate IS_GROUP_BY = new Predicate() { + @Override public boolean apply(DruidRel druidRel) { return druidRel.getQueryBuilder().getGrouping() != null; diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index 38f82c8213c7..d54a07f2f038 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -31,6 +31,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DirectDruidClient; import io.druid.client.DruidDataSource; import io.druid.client.DruidServer; import io.druid.client.ServerView; @@ -42,6 +43,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.query.QueryPlus; import io.druid.query.QuerySegmentWalker; import io.druid.query.TableDataSource; import io.druid.query.metadata.metadata.ColumnAnalysis; @@ -49,6 +51,7 @@ import io.druid.query.metadata.metadata.SegmentMetadataQuery; import io.druid.segment.column.ValueType; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; import io.druid.sql.calcite.table.RowSignature; @@ -81,6 +84,7 @@ public class DruidSchema extends AbstractSchema private final ViewManager viewManager; private final ExecutorService cacheExec; private final ConcurrentMap tables; + private final ServerConfig serverConfig; // For awaitInitialization. private final CountDownLatch initializationLatch = new CountDownLatch(1); @@ -99,7 +103,8 @@ public DruidSchema( final QuerySegmentWalker walker, final TimelineServerView serverView, final PlannerConfig config, - final ViewManager viewManager + final ViewManager viewManager, + final ServerConfig serverConfig ) { this.walker = Preconditions.checkNotNull(walker, "walker"); @@ -108,6 +113,7 @@ public DruidSchema( this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager"); this.cacheExec = ScheduledExecutors.fixed(1, "DruidSchema-Cache-%d"); this.tables = Maps.newConcurrentMap(); + this.serverConfig = serverConfig; } @LifecycleStart @@ -294,7 +300,7 @@ protected Multimap getFunctionMultimap() private DruidTable computeTable(final String dataSource) { - final SegmentMetadataQuery segmentMetadataQuery = new SegmentMetadataQuery( + SegmentMetadataQuery segmentMetadataQuery = new SegmentMetadataQuery( new TableDataSource(dataSource), null, null, @@ -305,7 +311,19 @@ private DruidTable computeTable(final String dataSource) true ); - final Sequence sequence = segmentMetadataQuery.run(walker, Maps.newHashMap()); + segmentMetadataQuery = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes( + segmentMetadataQuery, + serverConfig + ); + + final Sequence sequence = QueryPlus.wrap(segmentMetadataQuery) + .run( + walker, + DirectDruidClient.makeResponseContextForQuery( + segmentMetadataQuery, + System.currentTimeMillis() + ) + ); final List results = Sequences.toList(sequence, Lists.newArrayList()); if (results.isEmpty()) { return null; diff --git a/sql/src/main/java/io/druid/sql/calcite/view/InProcessViewManager.java b/sql/src/main/java/io/druid/sql/calcite/view/InProcessViewManager.java index 792c0f3b649d..d569e4c2fc13 100644 --- a/sql/src/main/java/io/druid/sql/calcite/view/InProcessViewManager.java +++ b/sql/src/main/java/io/druid/sql/calcite/view/InProcessViewManager.java @@ -42,6 +42,7 @@ public InProcessViewManager() this.views = new ConcurrentHashMap<>(); } + @Override public void createView(final PlannerFactory plannerFactory, final String viewName, final String viewSql) { final TableMacro oldValue = views.putIfAbsent(viewName, new DruidViewMacro(plannerFactory, viewSql)); @@ -50,6 +51,7 @@ public void createView(final PlannerFactory plannerFactory, final String viewNam } } + @Override public void alterView(final PlannerFactory plannerFactory, final String viewName, final String viewSql) { final TableMacro oldValue = views.replace(viewName, new DruidViewMacro(plannerFactory, viewSql)); @@ -58,6 +60,7 @@ public void alterView(final PlannerFactory plannerFactory, final String viewName } } + @Override public void dropView(final String viewName) { final TableMacro oldValue = views.remove(viewName); diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index 39259f3997ba..f2e7432e4629 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -19,13 +19,20 @@ package io.druid.sql.avatica; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import io.druid.java.util.common.Pair; import io.druid.server.DruidNode; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -57,11 +64,13 @@ import java.sql.Statement; import java.sql.Timestamp; import java.sql.Types; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.Set; +import java.util.concurrent.Executors; public class DruidAvaticaHandlerTest { @@ -76,7 +85,7 @@ public int getMaxConnections() @Override public int getMaxStatementsPerConnection() { - return 2; + return 4; } }; @@ -108,7 +117,10 @@ public void setUp() throws Exception ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final DruidAvaticaHandler handler = new DruidAvaticaHandler( - new DruidMeta(new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig), AVATICA_CONFIG), + new DruidMeta( + new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig, new ServerConfig()), + AVATICA_CONFIG + ), new DruidNode("dummy", "dummy", 1), new AvaticaMonitor() ); @@ -353,15 +365,48 @@ public void testDatabaseMetaDataColumns() throws Exception ); } + @Test(timeout = 30000) + public void testConcurrentQueries() throws Exception + { + final List> futures = new ArrayList<>(); + final ListeningExecutorService exec = MoreExecutors.listeningDecorator( + Executors.newFixedThreadPool(AVATICA_CONFIG.getMaxStatementsPerConnection()) + ); + for (int i = 0; i < 2000; i++) { + final String query = String.format("SELECT COUNT(*) + %s AS ci FROM foo", i); + futures.add( + exec.submit(() -> { + try ( + final Statement statement = client.createStatement(); + final ResultSet resultSet = statement.executeQuery(query) + ) { + final List> rows = getRows(resultSet); + return ((Number) Iterables.getOnlyElement(rows).get("ci")).intValue(); + } + catch (SQLException e) { + throw Throwables.propagate(e); + } + }) + ); + } + + final List integers = Futures.allAsList(futures).get(); + for (int i = 0; i < 2000; i++) { + Assert.assertEquals(i + 6, (int) integers.get(i)); + } + } + @Test public void testTooManyStatements() throws Exception { final Statement statement1 = client.createStatement(); final Statement statement2 = client.createStatement(); + final Statement statement3 = client.createStatement(); + final Statement statement4 = client.createStatement(); expectedException.expect(AvaticaClientRuntimeException.class); - expectedException.expectMessage("Too many open statements, limit is[2]"); - final Statement statement3 = client.createStatement(); + expectedException.expectMessage("Too many open statements, limit is[4]"); + final Statement statement5 = client.createStatement(); } @Test @@ -372,6 +417,11 @@ public void testNotTooManyStatementsWhenYouCloseThem() throws Exception client.createStatement().close(); client.createStatement().close(); client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); Assert.assertTrue(true); } diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java index 1f0ee59c4419..6cea2ff2e5fe 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java @@ -21,6 +21,7 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -65,7 +66,7 @@ public void setUp() throws Exception ) ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); - plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig); + plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig, new ServerConfig()); } @After diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 1ef1fe6d2097..4e4afd6e8fa0 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -81,6 +82,7 @@ import io.druid.query.topn.TopNQueryBuilder; import io.druid.segment.column.Column; import io.druid.segment.column.ValueType; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -169,35 +171,47 @@ public int getMaxQueryCount() private static final String LOS_ANGELES = "America/Los_Angeles"; private static final Map QUERY_CONTEXT_DEFAULT = ImmutableMap.of( - PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z" + PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final Map QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", - "skipEmptyBuckets", false + "skipEmptyBuckets", false, + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final Map QUERY_CONTEXT_NO_TOPN = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", - PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, "false" + PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, "false", + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final Map QUERY_CONTEXT_LOS_ANGELES = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", - PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES + PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES, + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); // Matches QUERY_CONTEXT_DEFAULT public static final Map TIMESERIES_CONTEXT_DEFAULT = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", - "skipEmptyBuckets", true + "skipEmptyBuckets", true, + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); // Matches QUERY_CONTEXT_LOS_ANGELES public static final Map TIMESERIES_CONTEXT_LOS_ANGELES = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES, - "skipEmptyBuckets", true + "skipEmptyBuckets", true, + QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final PagingSpec FIRST_PAGING_SPEC = new PagingSpec(null, 1000, true); @@ -4223,7 +4237,13 @@ private List getResults( final SchemaPlus rootSchema = Calcites.createRootSchema(druidSchema); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); - final PlannerFactory plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig); + final PlannerFactory plannerFactory = new PlannerFactory( + rootSchema, + walker, + operatorTable, + plannerConfig, + new ServerConfig() + ); viewManager.createView( plannerFactory, diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java index 392d19fbf748..88492861d0c9 100644 --- a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java @@ -28,6 +28,7 @@ import io.druid.java.util.common.Pair; import io.druid.query.QueryInterruptedException; import io.druid.query.ResourceLimitExceededException; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -77,7 +78,10 @@ public void setUp() throws Exception CalciteTests.createMockSchema(walker, plannerConfig) ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); - resource = new SqlResource(JSON_MAPPER, new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig)); + resource = new SqlResource( + JSON_MAPPER, + new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig, new ServerConfig()) + ); } @After diff --git a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java index 0d368186fc7a..982c5badba62 100644 --- a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java @@ -32,6 +32,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; @@ -150,7 +151,8 @@ public void setUp() throws Exception walker, new TestServerInventoryView(walker.getSegments()), PLANNER_CONFIG_DEFAULT, - new NoopViewManager() + new NoopViewManager(), + new ServerConfig() ); schema.start(); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index e3096d3a3b6e..bf8355058923 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -76,6 +76,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.server.initialization.ServerConfig; import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.SqlExtractionOperator; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -380,7 +381,8 @@ public static DruidSchema createMockSchema( walker, new TestServerInventoryView(walker.getSegments()), plannerConfig, - viewManager + viewManager, + new ServerConfig() ); schema.start(); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index c9b88af530a2..a96596555d6b 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -32,6 +32,7 @@ import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -110,8 +111,9 @@ public QueryRunner getQueryRunnerForIntervals( new QueryRunner() { @Override - public Sequence run(Query query, Map responseContext) + public Sequence run(QueryPlus queryPlus, Map responseContext) { + Query query = queryPlus.getQuery(); final VersionedIntervalTimeline timeline = getTimelineForTableDataSource(query); return makeBaseRunner( query, @@ -154,7 +156,7 @@ public SegmentDescriptor apply(final PartitionChunk chunk) } } ) - ).run(query, responseContext); + ).run(queryPlus, responseContext); } } ) diff --git a/sql/src/test/java/io/druid/sql/calcite/util/TestServerInventoryView.java b/sql/src/test/java/io/druid/sql/calcite/util/TestServerInventoryView.java index 193cc91c18c1..aa34d2411de0 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/TestServerInventoryView.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/TestServerInventoryView.java @@ -27,6 +27,7 @@ import io.druid.query.DataSource; import io.druid.query.QueryRunner; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; @@ -51,7 +52,7 @@ public TimelineLookup getTimeline(DataSource dataSource) @Override public void registerSegmentCallback(Executor exec, final SegmentCallback callback) { - final DruidServerMetadata dummyServer = new DruidServerMetadata("dummy", "dummy", 0, "dummy", "dummy", 0); + final DruidServerMetadata dummyServer = new DruidServerMetadata("dummy", "dummy", 0, ServerType.HISTORICAL, "dummy", 0); for (final DataSegment segment : segments) { exec.execute( From be31858736876f0f4de65d9af03f168a1b52d837 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 9 Jun 2017 01:31:05 +0900 Subject: [PATCH 05/15] Set the total shard number of NumberedShardSpec to 0 --- .../main/java/io/druid/indexing/common/task/IndexTask.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 7fad4f1aa4f3..06ff2b421a81 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -449,7 +449,9 @@ private static BiFunction getShardSpecCreateFunctio ) { if (useExtendableShardSpec) { - return NumberedShardSpec::new; + // 0 partitions means there's no core partitions. See NumberedPartitionChunk.isStart() and + // NumberedPartitionChunk.isEnd(). + return (shardId, notUsed) -> new NumberedShardSpec(shardId, 0); } else { if (numShards == null) { throw new ISE("numShards must not be null"); From 784eff7bd24aac94a1661b69af54acd1c1f87f08 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 16 Jun 2017 22:46:22 +0900 Subject: [PATCH 06/15] refactoring --- .../druid/indexing/common/task/IndexTask.java | 187 +++++++++++------- .../indexing/common/task/IndexTaskTest.java | 178 +++++++++-------- 2 files changed, 209 insertions(+), 156 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 06ff2b421a81..4daf34b7c295 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -241,10 +241,9 @@ private ShardSpecs determineShardSpecs( final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - final Granularity queryGranularity = granularitySpec.getQueryGranularity(); final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent(); - // Guarenteed rollup means that this index task guarantees the 'perfect rollup' across the entire data set. + // Guaranteed rollup means that this index task guarantees the 'perfect rollup' across the entire data set. final boolean guaranteedRollup = tuningConfig.isForceGuaranteedRollup() && !tuningConfig.isForceExtendableShardSpecs() && !ioConfig.isAppendToExisting(); @@ -254,39 +253,133 @@ private ShardSpecs determineShardSpecs( // if we were given number of shards per interval and the intervals, we don't need to scan the data if (!determineNumPartitions && !determineIntervals) { log.info("Skipping determine partition scan"); + return createShardSpecsWithoutDataScan(jsonMapper, granularitySpec, tuningConfig, useExtendableShardSpec); + } + + // determine intervals containing data and prime HLL collectors + return createShardSpecsFromData( + jsonMapper, + ingestionSchema, + firehoseFactory, + firehoseTempDir, + granularitySpec, + tuningConfig, + determineIntervals, + determineNumPartitions, + useExtendableShardSpec + ); + } + + private static ShardSpecs createShardSpecsWithoutDataScan( + ObjectMapper jsonMapper, + GranularitySpec granularitySpec, + IndexTuningConfig tuningConfig, + boolean useExtendableShardSpec + ) + { + final int numShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); + final BiFunction shardSpecCreateFn = getShardSpecCreateFunction( + useExtendableShardSpec, + numShards, + jsonMapper + ); + + final Map> intervalToShardSpecs = new HashMap<>(); + for (Interval interval : granularitySpec.bucketIntervals().get()) { + final List intervalShardSpecs = IntStream.range(0, numShards) + .mapToObj( + shardId -> shardSpecCreateFn.apply(shardId, numShards) + ) + .collect(Collectors.toList()); + intervalToShardSpecs.put(interval, intervalShardSpecs); + } + + if (useExtendableShardSpec) { + return createExtendableShardSpecs(intervalToShardSpecs); + } else { + return createNonExtendableShardSpecs(intervalToShardSpecs); + } + } + + private static ShardSpecs createShardSpecsFromData( + ObjectMapper jsonMapper, + IndexIngestionSpec ingestionSchema, + FirehoseFactory firehoseFactory, + File firehoseTempDir, + GranularitySpec granularitySpec, + IndexTuningConfig tuningConfig, + boolean determineIntervals, + boolean determineNumPartitions, + boolean useExtendableShardSpec + ) throws IOException + { + log.info("Determining intervals and shardSpecs"); + long determineShardSpecsStartMillis = System.currentTimeMillis(); + + final Map> hllCollectors = collectIntervalsAndShardSpecs( + jsonMapper, + ingestionSchema, + firehoseFactory, + firehoseTempDir, + granularitySpec, + determineIntervals, + determineNumPartitions + ); + + final Map> intervalToShardSpecs = new HashMap<>(); + final int defaultNumShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); + for (final Map.Entry> entry : hllCollectors.entrySet()) { + final Interval interval = entry.getKey(); + final Optional collector = entry.getValue(); + + final int numShards; + if (determineNumPartitions) { + final long numRows = new Double(collector.get().estimateCardinality()).longValue(); + numShards = (int) Math.ceil((double) numRows / tuningConfig.getTargetPartitionSize()); + log.info("Estimated [%,d] rows of data for interval [%s], creating [%,d] shards", numRows, interval, numShards); + } else { + numShards = defaultNumShards; + log.info("Creating [%,d] shards for interval [%s]", numShards, interval); + } - final int numShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); final BiFunction shardSpecCreateFn = getShardSpecCreateFunction( useExtendableShardSpec, numShards, jsonMapper ); - final Map> intervalToShardSpecs = new HashMap<>(); - for (Interval interval : granularitySpec.bucketIntervals().get()) { - final List intervalShardSpecs = IntStream.range(0, numShards) - .mapToObj( - shardId -> shardSpecCreateFn.apply(shardId, numShards) - ) - .collect(Collectors.toList()); - intervalToShardSpecs.put(interval, intervalShardSpecs); - } + final List intervalShardSpecs = IntStream.range(0, numShards) + .mapToObj( + shardId -> shardSpecCreateFn.apply(shardId, numShards) + ).collect(Collectors.toList()); - if (useExtendableShardSpec) { - return createExtendableShardSpecs(intervalToShardSpecs); - } else { - return createNonExtendableShardSpecs(intervalToShardSpecs); - } + intervalToShardSpecs.put(interval, intervalShardSpecs); + } + log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); + + if (useExtendableShardSpec) { + return createExtendableShardSpecs(intervalToShardSpecs); + } else { + return createNonExtendableShardSpecs(intervalToShardSpecs); } + } - // determine intervals containing data and prime HLL collectors + private static Map> collectIntervalsAndShardSpecs( + ObjectMapper jsonMapper, + IndexIngestionSpec ingestionSchema, + FirehoseFactory firehoseFactory, + File firehoseTempDir, + GranularitySpec granularitySpec, + boolean determineIntervals, + boolean determineNumPartitions + ) throws IOException + { final Map> hllCollectors = new TreeMap<>( Comparators.intervalsByStartThenEnd() ); int thrownAway = 0; + final Granularity queryGranularity = granularitySpec.getQueryGranularity(); - log.info("Determining intervals and shardSpecs"); - long determineShardSpecsStartMillis = System.currentTimeMillis(); try ( final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser(), firehoseTempDir) ) { @@ -334,43 +427,7 @@ private ShardSpecs determineShardSpecs( if (thrownAway > 0) { log.warn("Unable to to find a matching interval for [%,d] events", thrownAway); } - - final Map> intervalToShardSpecs = new HashMap<>(); - final int defaultNumShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); - for (final Map.Entry> entry : hllCollectors.entrySet()) { - final Interval interval = entry.getKey(); - final Optional collector = entry.getValue(); - - final int numShards; - if (determineNumPartitions) { - final long numRows = new Double(collector.get().estimateCardinality()).longValue(); - numShards = (int) Math.ceil((double) numRows / tuningConfig.getTargetPartitionSize()); - log.info("Estimated [%,d] rows of data for interval [%s], creating [%,d] shards", numRows, interval, numShards); - } else { - numShards = defaultNumShards; - log.info("Creating [%,d] shards for interval [%s]", numShards, interval); - } - - final BiFunction shardSpecCreateFn = getShardSpecCreateFunction( - useExtendableShardSpec, - numShards, - jsonMapper - ); - - final List intervalShardSpecs = IntStream.range(0, numShards) - .mapToObj( - shardId -> shardSpecCreateFn.apply(shardId, numShards) - ).collect(Collectors.toList()); - - intervalToShardSpecs.put(interval, intervalShardSpecs); - } - log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); - - if (useExtendableShardSpec) { - return createExtendableShardSpecs(intervalToShardSpecs); - } else { - return createNonExtendableShardSpecs(intervalToShardSpecs); - } + return hllCollectors; } private static ShardSpecs createNonExtendableShardSpecs(Map> intervalToShardSpecs) @@ -403,17 +460,15 @@ public void updateShardSpec(Interval interval) private static ShardSpecs createExtendableShardSpecs(Map> intervalToShardSpec) { - return new ShardSpecs() - { - private final Map shardSpecMap = new HashMap<>(intervalToShardSpec.size()); + final Map shardSpecMap = new HashMap<>(intervalToShardSpec.size()); - { - intervalToShardSpec.forEach((interval, shardSpecs) -> { - Preconditions.checkState(shardSpecs.size() == 1); - shardSpecMap.put(interval, shardSpecs.get(0)); - }); - } + intervalToShardSpec.forEach((interval, shardSpecs) -> { + Preconditions.checkState(shardSpecs.size() == 1); + shardSpecMap.put(interval, shardSpecs.get(0)); + }); + return new ShardSpecs() + { @Override public Collection getIntervals() { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index d58fec5dbf07..4c1e48f688da 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -37,6 +37,7 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.aggregation.AggregatorFactory; @@ -96,7 +97,7 @@ public class IndexTaskTest 0 ); - private final IndexSpec indexSpec; + private static final IndexSpec indexSpec = new IndexSpec(); private final ObjectMapper jsonMapper; private IndexMerger indexMerger; private IndexMergerV9 indexMergerV9; @@ -105,7 +106,6 @@ public class IndexTaskTest public IndexTaskTest() { - indexSpec = new IndexSpec(); TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); indexMerger = testUtils.getTestIndexMerger(); @@ -129,7 +129,13 @@ public void testDeterminePartitions() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, null, 2, null, null, false, false, true), + createIngestionSpec( + tmpDir, + null, + null, + createTuningConfig(2, null, false, true), + false + ), null, jsonMapper ); @@ -167,7 +173,13 @@ public void testForceExtendableShardSpecs() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, null, 2, null, null, true, false, false), + createIngestionSpec( + tmpDir, + null, + null, + createTuningConfig(2, null, true, false), + false + ), null, jsonMapper ); @@ -210,12 +222,8 @@ public void testWithArbitraryGranularity() throws Exception Granularities.MINUTE, Collections.singletonList(new Interval("2014/2015")) ), - 10, - null, - null, - false, - false, - true + createTuningConfig(10, null, false, true), + false ), null, jsonMapper @@ -249,12 +257,8 @@ public void testIntervalBucketing() throws Exception Granularities.HOUR, Collections.singletonList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z")) ), - 50, - null, - null, - false, - false, - true + createTuningConfig(50, null, false, true), + false ), null, jsonMapper @@ -280,7 +284,13 @@ public void testNumShardsProvided() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, null, null, 1, null, false, false, true), + createIngestionSpec( + tmpDir, + null, + null, + createTuningConfig(null, 1, false, true), + false + ), null, jsonMapper ); @@ -311,7 +321,13 @@ public void testAppendToExisting() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec(tmpDir, null, null, 2, null, null, false, true, false), + createIngestionSpec( + tmpDir, + null, + null, + createTuningConfig(2, null, false, false), + true + ), null, jsonMapper ); @@ -355,12 +371,8 @@ public void testIntervalNotSpecified() throws Exception Granularities.MINUTE, null ), - 2, - null, - null, - false, - false, - true + createTuningConfig(2, null, false, true), + false ), null, jsonMapper @@ -420,12 +432,8 @@ public void testCSVFileWithHeader() throws Exception 0 ), null, - 2, - null, - null, - false, - false, - true + createTuningConfig(2, null, false, true), + false ), null, jsonMapper @@ -474,12 +482,8 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception 0 ), null, - 2, - null, - null, - false, - false, - true + createTuningConfig(2, null, false, true), + false ), null, jsonMapper @@ -523,12 +527,7 @@ public void testWithSmallMaxTotalRows() throws Exception Granularities.MINUTE, null ), - 2, - null, - 2, - 2, - false, - false, + createTuningConfig(2, 2, 2, null, false, false), false ), null, @@ -636,39 +635,8 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( File baseDir, ParseSpec parseSpec, GranularitySpec granularitySpec, - Integer targetPartitionSize, - Integer numShards, - Integer maxTotalRows, - boolean forceExtendableShardSpecs, - boolean appendToExisting, - boolean forceGuaranteedRollup - ) - { - return createIngestionSpec( - baseDir, - parseSpec, - granularitySpec, - targetPartitionSize, - numShards, - 1, - maxTotalRows, - forceExtendableShardSpecs, - appendToExisting, - forceGuaranteedRollup - ); - } - - private IndexTask.IndexIngestionSpec createIngestionSpec( - File baseDir, - ParseSpec parseSpec, - GranularitySpec granularitySpec, - Integer targetPartitionSize, - Integer numShards, - Integer maxRowsInMemory, - Integer maxTotalRows, - boolean forceExtendableShardSpecs, - boolean appendToExisting, - boolean forceGuaranteedRollup + IndexTuningConfig tuningConfig, + boolean appendToExisting ) { return new IndexTask.IndexIngestionSpec( @@ -696,22 +664,52 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( baseDir, "druid*", null - ), appendToExisting + ), + appendToExisting ), - new IndexTask.IndexTuningConfig( - targetPartitionSize, - maxRowsInMemory, - maxTotalRows, - null, - numShards, - indexSpec, - null, - true, - forceExtendableShardSpecs, - forceGuaranteedRollup, - null, - null - ) + tuningConfig + ); + } + + private static IndexTuningConfig createTuningConfig( + Integer targetPartitionSize, + Integer numShards, + boolean forceExtendableShardSpecs, + boolean forceGuaranteedRollup + ) + { + return createTuningConfig( + targetPartitionSize, + 1, + null, + numShards, + forceExtendableShardSpecs, + forceGuaranteedRollup + ); + } + + private static IndexTuningConfig createTuningConfig( + Integer targetPartitionSize, + Integer maxRowsInMemory, + Integer maxTotalRows, + Integer numShards, + boolean forceExtendableShardSpecs, + boolean forceGuaranteedRollup + ) + { + return new IndexTask.IndexTuningConfig( + targetPartitionSize, + maxRowsInMemory, + maxTotalRows, + null, + numShards, + indexSpec, + null, + true, + forceExtendableShardSpecs, + forceGuaranteedRollup, + null, + null ); } } From d8ecc48323e84bb7e899f3d245d9814e130a833a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 20 Jun 2017 09:57:32 +0900 Subject: [PATCH 07/15] Address comments --- docs/content/design/index.md | 49 +++++---- docs/content/ingestion/tasks.md | 12 ++- .../druid/indexing/common/task/IndexTask.java | 60 ++++++----- .../indexing/common/task/IndexTaskTest.java | 99 +++++++++++++++++++ 4 files changed, 175 insertions(+), 45 deletions(-) diff --git a/docs/content/design/index.md b/docs/content/design/index.md index fb48c6dcf6b8..f4399be1ec39 100644 --- a/docs/content/design/index.md +++ b/docs/content/design/index.md @@ -32,6 +32,30 @@ They each represent an axis of the data that we’ve chosen to slice across. Metrics are usually numeric values, and computations include operations such as count, sum, and mean. Also known as measures in standard OLAP terminology. +## Sharding the Data + +Druid shards are called `segments` and Druid always first shards data by time. In our compacted data set, we can create two segments, one for each hour of data. + +For example: + +Segment `sampleData_2011-01-01T01:00:00:00Z_2011-01-01T02:00:00:00Z_v1_0` contains + + 2011-01-01T01:00:00Z ultratrimfast.com google.com Male USA 1800 25 15.70 + 2011-01-01T01:00:00Z bieberfever.com google.com Male USA 2912 42 29.18 + + +Segment `sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_0` contains + + 2011-01-01T02:00:00Z ultratrimfast.com google.com Male UK 1953 17 17.31 + 2011-01-01T02:00:00Z bieberfever.com google.com Male UK 3194 170 34.01 + +Segments are self-contained containers for the time interval of data they hold. Segments +contain data stored in compressed column orientations, along with the indexes for those columns. Druid queries only understand how to +scan segments. + +Segments are uniquely identified by a datasource, interval, version, and an optional partition number. +Examining our example segments, the segments are named following this convention: `dataSource_interval_version_partitionNumber` + ## Roll-up The individual events in our example data set are not very interesting because there may be trillions of such events. @@ -56,30 +80,15 @@ This storage reduction does come at a cost; as we roll up data, we lose the abil the rollup granularity is the minimum granularity you will be able to explore data at and events are floored to this granularity. Hence, Druid ingestion specs define this granularity as the `queryGranularity` of the data. The lowest supported `queryGranularity` is millisecond. -## Sharding the Data - -Druid shards are called `segments` and Druid always first shards data by time. In our compacted data set, we can create two segments, one for each hour of data. +### Roll-up modes -For example: - -Segment `sampleData_2011-01-01T01:00:00:00Z_2011-01-01T02:00:00:00Z_v1_0` contains - - 2011-01-01T01:00:00Z ultratrimfast.com google.com Male USA 1800 25 15.70 - 2011-01-01T01:00:00Z bieberfever.com google.com Male USA 2912 42 29.18 +Druid supports two roll-up modes, i.e., _perfect roll-up_ and _best-effort roll-up_. In the perfect roll-up mode, Druid guarantees that input data are perfectly aggregated at ingestion time. Meanwhile, in the best-effort roll-up, input data might not be perfectly aggregated and thus there can be multiple segments holding the rows which should belong to the same segment with the perfect roll-up since they have the same dimension value and their timestamps fall into the same interval. +The perfect roll-up mode encompasses an additional preprocessing step to determine intervals and shardSpecs before actual data ingestion if they are not specified in the ingestionSpec. This preprocessing step usually scans the entire input data which might increase the ingestion time. The [Hadoop indexing task](./ingestion/batch-ingestion.html) always runs with this perfect roll-up mode. -Segment `sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_0` contains - - 2011-01-01T02:00:00Z ultratrimfast.com google.com Male UK 1953 17 17.31 - 2011-01-01T02:00:00Z bieberfever.com google.com Male UK 3194 170 34.01 - -Segments are self-contained containers for the time interval of data they hold. Segments -contain data stored in compressed column orientations, along with the indexes for those columns. Druid queries only understand how to -scan segments. - -Segments are uniquely identified by a datasource, interval, version, and an optional partition number. -Examining our example segments, the segments are named following this convention: `dataSource_interval_version_partitionNumber` +On the contrary, the best-effort roll-up mode doesn't require any preprocessing step, but the size of ingested data might be larger than that of the perfect roll-up. All types of [streaming indexing (i.e., realtime index task, kafka indexing service, ...)](./ingestion/stream-ingestion.html) run with this mode. +Finally, the [native index task](./ingestion/tasks.html) supports both modes and you can choose either one which fits to your application. ## Indexing the Data diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 835c374ecc9a..86671b11cdb3 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -120,7 +120,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| |buildV9Directly|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|true|no| |forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| -|forceGuaranteedRollup|Forces guaranteeing the perfect rollup. The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with `appendToExisting` of IOConfig and `forceExtendableShardSpecs`.|false|no| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](./design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with `appendToExisting` of IOConfig and `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| |publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| @@ -151,6 +151,16 @@ For Roaring bitmaps: |type|String|Must be `roaring`.|yes| |compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)| +#### Segment publishing modes + +While ingesting data using the Index task, it creates segments from the input data and publishes them. For segment publishing, the Index task supports two segment publishing modes, i.e., _bulk publishing mode_ and _incremental publishing mode_ for [perfect rollup and best-effort rollup](./design/index.html), respectively. + +In the bulk publishing mode, every segment is published at the very end of the index task. Until then, created segments are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a problem due to limited storage capacity, and is not recommended to use in production. + +On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to ingest remaining data. + +To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with `forceExtendableShardSpecs` of TuningConfig and `appendToExisting` of IOConfig together. + Segment Merging Tasks --------------------- diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 4daf34b7c295..f1b7d06e2757 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -215,6 +215,16 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } } + private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) + { + Preconditions.checkState( + !(tuningConfig.isForceGuaranteedRollup() && + (tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting())), + "Perfect rollup cannot be guaranteed with extendable shardSpecs" + ); + return tuningConfig.isForceGuaranteedRollup(); + } + /** * Determines intervals and shardSpecs for input data. This method first checks that it must determine intervals and * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. @@ -244,33 +254,31 @@ private ShardSpecs determineShardSpecs( final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent(); // Guaranteed rollup means that this index task guarantees the 'perfect rollup' across the entire data set. - final boolean guaranteedRollup = tuningConfig.isForceGuaranteedRollup() && - !tuningConfig.isForceExtendableShardSpecs() && - !ioConfig.isAppendToExisting(); + final boolean guaranteedRollup = isGuaranteedRollup(ioConfig, tuningConfig); final boolean determineNumPartitions = tuningConfig.getNumShards() == null && guaranteedRollup; final boolean useExtendableShardSpec = !guaranteedRollup; // if we were given number of shards per interval and the intervals, we don't need to scan the data if (!determineNumPartitions && !determineIntervals) { log.info("Skipping determine partition scan"); - return createShardSpecsWithoutDataScan(jsonMapper, granularitySpec, tuningConfig, useExtendableShardSpec); + return createShardSpecWithoutInputScan(jsonMapper, granularitySpec, tuningConfig, useExtendableShardSpec); + } else { + // determine intervals containing data and prime HLL collectors + return createShardSpecsFromInput( + jsonMapper, + ingestionSchema, + firehoseFactory, + firehoseTempDir, + granularitySpec, + tuningConfig, + determineIntervals, + determineNumPartitions, + useExtendableShardSpec + ); } - - // determine intervals containing data and prime HLL collectors - return createShardSpecsFromData( - jsonMapper, - ingestionSchema, - firehoseFactory, - firehoseTempDir, - granularitySpec, - tuningConfig, - determineIntervals, - determineNumPartitions, - useExtendableShardSpec - ); } - private static ShardSpecs createShardSpecsWithoutDataScan( + private static ShardSpecs createShardSpecWithoutInputScan( ObjectMapper jsonMapper, GranularitySpec granularitySpec, IndexTuningConfig tuningConfig, @@ -301,7 +309,7 @@ private static ShardSpecs createShardSpecsWithoutDataScan( } } - private static ShardSpecs createShardSpecsFromData( + private static ShardSpecs createShardSpecsFromInput( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, @@ -570,6 +578,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final int maxRowsInSegment = tuningConfig.getTargetPartitionSize() == null ? Integer.MAX_VALUE : tuningConfig.getTargetPartitionSize(); + final boolean isGuaranteedRollup = isGuaranteedRollup(ioConfig, tuningConfig); final SegmentAllocator segmentAllocator; if (ioConfig.isAppendToExisting()) { @@ -634,8 +643,10 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); if (addResult.isOk()) { - if (addResult.getNumRowsInSegment() >= maxRowsInSegment || - addResult.getTotalNumRowsInAppenderator() >= maxRowsInAppenderator) { + // incremental segment publishment is allowed only when rollup don't have to be perfect. + if (!isGuaranteedRollup && + (addResult.getNumRowsInSegment() >= maxRowsInSegment || + addResult.getTotalNumRowsInAppenderator() >= maxRowsInAppenderator)) { // There can be some segments waiting for being published even though any rows won't be added to them. // If those segments are not published here, the available space in appenderator will be kept to be small // which makes the size of segments smaller. @@ -973,9 +984,10 @@ private IndexTuningConfig( this.publishTimeout = publishTimeout == null ? DEFAULT_PUBLISH_TIMEOUT : publishTimeout; this.basePersistDirectory = basePersistDirectory; - if (this.forceGuaranteedRollup && this.forceExtendableShardSpecs) { - log.warn("Perfect rollup is not guaranteed with extendable shardSpecs. forceGuaranteedRollup flag is ignored."); - } + Preconditions.checkArgument( + !(this.forceExtendableShardSpecs && this.forceGuaranteedRollup), + "Perfect rollup cannot be guaranteed with extendable shardSpecs" + ); } public IndexTuningConfig withBasePersistDirectory(File dir) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 4c1e48f688da..17fa9e9b1eee 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -550,6 +550,105 @@ public void testWithSmallMaxTotalRows() throws Exception } } + @Test + public void testPerfectRollup() throws Exception + { + File tmpDir = temporaryFolder.newFolder(); + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + populateRollupTestData(tmpFile); + + IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + tmpDir, + null, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.DAY, + true, + null + ), + createTuningConfig(3, 2, 2, null, false, true), + false + ), + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + + Assert.assertEquals(3, segments.size()); + + for (int i = 0; i < 3; i++) { + final DataSegment segment = segments.get(i); + final Interval expectedInterval = new Interval("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"); + + Assert.assertEquals("test", segment.getDataSource()); + Assert.assertEquals(expectedInterval, segment.getInterval()); + Assert.assertTrue(segment.getShardSpec().getClass().equals(HashBasedNumberedShardSpec.class)); + Assert.assertEquals(i, segment.getShardSpec().getPartitionNum()); + } + } + + @Test + public void testBestEffortRollup() throws Exception + { + File tmpDir = temporaryFolder.newFolder(); + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + populateRollupTestData(tmpFile); + + IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + tmpDir, + null, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.DAY, + true, + null + ), + createTuningConfig(3, 2, 2, null, false, false), + false + ), + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + + Assert.assertEquals(5, segments.size()); + + for (int i = 0; i < 5; i++) { + final DataSegment segment = segments.get(i); + final Interval expectedInterval = new Interval("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"); + + Assert.assertEquals("test", segment.getDataSource()); + Assert.assertEquals(expectedInterval, segment.getInterval()); + Assert.assertTrue(segment.getShardSpec().getClass().equals(NumberedShardSpec.class)); + Assert.assertEquals(i, segment.getShardSpec().getPartitionNum()); + } + } + + private static void populateRollupTestData(File tmpFile) throws IOException + { + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("2014-01-01T00:00:10Z,a,1\n"); + writer.write("2014-01-01T01:00:20Z,a,1\n"); + writer.write("2014-01-01T00:00:10Z,b,2\n"); + writer.write("2014-01-01T00:00:10Z,c,3\n"); + writer.write("2014-01-01T01:00:20Z,b,2\n"); + writer.write("2014-01-01T02:00:30Z,a,1\n"); + writer.write("2014-01-01T02:00:30Z,b,2\n"); + writer.write("2014-01-01T01:00:20Z,c,3\n"); + writer.write("2014-01-01T02:00:30Z,c,3\n"); + } + } + private final List runTask(final IndexTask indexTask) throws Exception { final List segments = Lists.newArrayList(); From b27e7f69991323151ed41d5dda6631d6a70bd312 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 20 Jun 2017 21:43:46 +0900 Subject: [PATCH 08/15] Fix tests --- .../java/io/druid/indexing/common/task/TaskSerdeTest.java | 4 ++-- .../java/io/druid/indexing/overlord/TaskLifecycleTest.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 80ca2faccbb9..28f15635f30d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -186,7 +186,7 @@ public void testIndexTaskSerde() throws Exception jsonMapper ), new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), - new IndexTask.IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, true, null, null) + new IndexTask.IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null) ), null, jsonMapper @@ -250,7 +250,7 @@ public void testIndexTaskwithResourceSerde() throws Exception jsonMapper ), new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true, null, null) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) ), null, jsonMapper diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index d8abc041c421..bfd37482997c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -656,7 +656,7 @@ public void testIndexTask() throws Exception mapper ), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false), false), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true, null, null) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) ), null, MAPPER @@ -714,7 +714,7 @@ public void testIndexTaskFailure() throws Exception mapper ), new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory(), false), - new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, true, null, null) + new IndexTask.IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) ), null, MAPPER From 70cf3afdcfd65d331186c87c691e9592c6119173 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 27 Jun 2017 16:31:07 +0900 Subject: [PATCH 09/15] Address comments --- docs/content/ingestion/tasks.md | 4 ++-- .../druid/indexing/common/task/IndexTask.java | 7 +----- .../realtime/appenderator/Appenderator.java | 10 -------- .../appenderator/AppenderatorImpl.java | 23 ++++++++++--------- 4 files changed, 15 insertions(+), 29 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 61b93b5fef38..76f1268dd4f2 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -119,7 +119,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| |forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| -|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](./design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with `appendToExisting` of IOConfig and `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](./design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| |publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| @@ -158,7 +158,7 @@ In the bulk publishing mode, every segment is published at the very end of the i On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to ingest remaining data. -To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with `forceExtendableShardSpecs` of TuningConfig and `appendToExisting` of IOConfig together. +To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig. Segment Merging Tasks --------------------- diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index d535486de362..fd8246e34c96 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -662,7 +662,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null // the below code is needed to update the total number of rows added to the appenderator so far. // See AppenderatorDriver.registerHandoff() and Appenderator.drop(). // A hard-coded timeout is used here because the below get() is expected to return immediately. - driver.registerHandoff(published).get(1000, TimeUnit.MILLISECONDS); + driver.registerHandoff(published).get(30, TimeUnit.SECONDS); } } else { throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); @@ -816,11 +816,6 @@ public IndexIngestionSpec( this.dataSchema = dataSchema; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig == null ? new IndexTuningConfig() : tuningConfig; - - if (this.ioConfig.isAppendToExisting() && this.tuningConfig.isForceGuaranteedRollup()) { - log.warn("Perfect rollup is not guaranteed when appending to existing data sources." - + " forceGuaranteedRollup flag is ignored."); - } } @Override diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index 45caa34d9af4..19137bc37558 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -61,8 +61,6 @@ public interface Appenderator extends QuerySegmentWalker, Closeable * This method may trigger a {@link #persistAll(Committer)} using the supplied Committer. If it does this, the * Committer is guaranteed to be *created* synchronously with the call to add, but will actually be used * asynchronously. - *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifier the segment into which this row should be added * @param row the row to add @@ -103,8 +101,6 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * Drop all in-memory and on-disk data, and forget any previously-remembered commit metadata. This could be useful if, * for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been * cleared. This may take some time, since all pending persists must finish first. - * - * The add, clear, persist, persistAll, and push methods should all be called from the same thread. */ void clear() throws InterruptedException; @@ -127,8 +123,6 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * somewhat durable, e.g. the machine's local disk. The Committer will be made synchronously with the call to * persist, but will actually be used asynchronously. Any metadata returned by the committer will be associated with * the data persisted to disk. - *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifiers segment identifiers to be persisted * @param committer a committer associated with all data that has been added to segments of the given identifiers so @@ -144,8 +138,6 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * machine's local disk. The Committer will be made synchronously with the call to persistAll, but will actually * be used asynchronously. Any metadata returned by the committer will be associated with the data persisted to * disk. - *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param committer a committer associated with all data that has been added so far * @@ -161,8 +153,6 @@ default ListenableFuture persistAll(Committer committer) * {@link #persist(Collection, Committer)} using the provided Committer. *

* After this method is called, you cannot add new data to any segments that were previously under construction. - *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifiers list of segments to push * @param committer a committer associated with all data that has been added so far diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index d780fa99bbbf..7b79ce421434 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -89,6 +89,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; /** */ @@ -114,6 +115,9 @@ public class AppenderatorImpl implements Appenderator ); private final QuerySegmentWalker texasRanger; + // This variable updated in add(), persist(), and drop() + private final AtomicInteger rowsCurrentlyInMemory = new AtomicInteger(); + private final AtomicInteger totalRows = new AtomicInteger(); private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; @@ -121,9 +125,6 @@ public class AppenderatorImpl implements Appenderator private volatile FileLock basePersistDirLock = null; private volatile FileChannel basePersistDirLockChannel = null; - private int rowsCurrentlyInMemory; - private int totalRows; - public AppenderatorImpl( DataSchema schema, AppenderatorConfig tuningConfig, @@ -216,12 +217,12 @@ public int add( } final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd; - rowsCurrentlyInMemory += numAddedRows; - totalRows += numAddedRows; + rowsCurrentlyInMemory.addAndGet(numAddedRows); + totalRows.addAndGet(numAddedRows); if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush - || rowsCurrentlyInMemory >= tuningConfig.getMaxRowsInMemory()) { + || rowsCurrentlyInMemory.get() >= tuningConfig.getMaxRowsInMemory()) { // persistAll clears rowsCurrentlyInMemory, no need to update it. persistAll(committerSupplier.get()); } @@ -250,13 +251,13 @@ public int getRowCount(final SegmentIdentifier identifier) @Override public int getTotalRowCount() { - return totalRows; + return totalRows.get(); } @VisibleForTesting int getRowsInMemory() { - return rowsCurrentlyInMemory; + return rowsCurrentlyInMemory.get(); } private Sink getOrCreateSink(final SegmentIdentifier identifier) @@ -444,7 +445,7 @@ public String apply(Map.Entry entry) resetNextFlush(); // NB: The rows are still in memory until they're done persisting, but we only count rows in active indexes. - rowsCurrentlyInMemory -= numPersistedRows; + rowsCurrentlyInMemory.addAndGet(-numPersistedRows); return future; } @@ -879,8 +880,8 @@ private ListenableFuture abandonSegment( droppingSinks.add(identifier); // Decrement this sink's rows from rowsCurrentlyInMemory (we only count active sinks). - rowsCurrentlyInMemory -= sink.getNumRowsInMemory(); - totalRows -= sink.getNumRows(); + rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory()); + totalRows.addAndGet(-sink.getNumRows()); // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( From 3df551c0191f13e358619cf54c68730c88d4805a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 30 Jun 2017 11:40:08 +0900 Subject: [PATCH 10/15] Fix sync problem of committer and retry push only --- .../realtime/appenderator/Appenderator.java | 10 ++ .../appenderator/AppenderatorDriver.java | 103 ++++++++---------- .../appenderator/AppenderatorImpl.java | 15 ++- .../AppenderatorDriverFailTest.java | 48 ++------ .../appenderator/AppenderatorTest.java | 14 +-- .../appenderator/AppenderatorTester.java | 22 +++- 6 files changed, 103 insertions(+), 109 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index 19137bc37558..45caa34d9af4 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -61,6 +61,8 @@ public interface Appenderator extends QuerySegmentWalker, Closeable * This method may trigger a {@link #persistAll(Committer)} using the supplied Committer. If it does this, the * Committer is guaranteed to be *created* synchronously with the call to add, but will actually be used * asynchronously. + *

+ * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifier the segment into which this row should be added * @param row the row to add @@ -101,6 +103,8 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * Drop all in-memory and on-disk data, and forget any previously-remembered commit metadata. This could be useful if, * for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been * cleared. This may take some time, since all pending persists must finish first. + * + * The add, clear, persist, persistAll, and push methods should all be called from the same thread. */ void clear() throws InterruptedException; @@ -123,6 +127,8 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * somewhat durable, e.g. the machine's local disk. The Committer will be made synchronously with the call to * persist, but will actually be used asynchronously. Any metadata returned by the committer will be associated with * the data persisted to disk. + *

+ * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifiers segment identifiers to be persisted * @param committer a committer associated with all data that has been added to segments of the given identifiers so @@ -138,6 +144,8 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * machine's local disk. The Committer will be made synchronously with the call to persistAll, but will actually * be used asynchronously. Any metadata returned by the committer will be associated with the data persisted to * disk. + *

+ * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param committer a committer associated with all data that has been added so far * @@ -153,6 +161,8 @@ default ListenableFuture persistAll(Committer committer) * {@link #persist(Collection, Committer)} using the provided Committer. *

* After this method is called, you cannot add new data to any segments that were previously under construction. + *

+ * The add, clear, persist, persistAll, and push methods should all be called from the same thread. * * @param identifiers list of segments to push * @param committer a committer associated with all data that has been added so far diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java index d3c26eb2f3d7..81d5642c3838 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorDriver.java @@ -57,7 +57,6 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Objects; -import java.util.Random; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; @@ -542,67 +541,59 @@ private ListenableFuture publish( final List segmentIdentifiers ) { - return publishExecutor.submit( - () -> { - long nTry = 0; - while (true) { - try { - log.info("Pushing segments: [%s]", Joiner.on(", ").join(segmentIdentifiers)); - final SegmentsAndMetadata segmentsAndMetadata = appenderator.push(segmentIdentifiers, wrappedCommitter) - .get(); - - // Sanity check - final Set pushedSegments = segmentsAndMetadata.getSegments().stream() - .map(SegmentIdentifier::fromDataSegment) - .collect(Collectors.toSet()); - if (!pushedSegments.equals(Sets.newHashSet(segmentIdentifiers))) { - throw new ISE( - "WTF?! Pushed different segments than requested. Pushed[%s], requested[%s].", - pushedSegments, - segmentIdentifiers - ); - } + log.info("Pushing segments: [%s]", Joiner.on(", ").join(segmentIdentifiers)); - log.info( - "Publishing segments with commitMetadata[%s]: [%s]", - segmentsAndMetadata.getCommitMetadata(), - Joiner.on(", ").join(segmentsAndMetadata.getSegments()) + return Futures.transform( + appenderator.push(segmentIdentifiers, wrappedCommitter), + (Function) segmentsAndMetadata -> { + // Sanity check + final Set pushedSegments = segmentsAndMetadata.getSegments().stream() + .map(SegmentIdentifier::fromDataSegment) + .collect(Collectors.toSet()); + if (!pushedSegments.equals(Sets.newHashSet(segmentIdentifiers))) { + throw new ISE( + "WTF?! Pushed different segments than requested. Pushed[%s], requested[%s].", + pushedSegments, + segmentIdentifiers + ); + } + + if (segmentsAndMetadata.getSegments().isEmpty()) { + log.info("Nothing to publish, skipping publish step."); + } else { + log.info( + "Publishing segments with commitMetadata[%s]: [%s]", + segmentsAndMetadata.getCommitMetadata(), + Joiner.on(", ").join(segmentsAndMetadata.getSegments()) + ); + + try { + final boolean published = publisher.publishSegments( + ImmutableSet.copyOf(segmentsAndMetadata.getSegments()), + ((AppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() ); - if (segmentsAndMetadata.getSegments().isEmpty()) { - log.info("Nothing to publish, skipping publish step."); + if (published) { + log.info("Published segments."); } else { - final boolean published = publisher.publishSegments( - ImmutableSet.copyOf(segmentsAndMetadata.getSegments()), - ((AppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() - ); - - if (published) { - log.info("Published segments, awaiting handoff."); + log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); + if (usedSegmentChecker.findUsedSegments(pushedSegments) + .equals(Sets.newHashSet(segmentsAndMetadata.getSegments()))) { + log.info("Our segments really do exist, awaiting handoff."); } else { - log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); - if (usedSegmentChecker.findUsedSegments(pushedSegments) - .equals(Sets.newHashSet(segmentsAndMetadata.getSegments()))) { - log.info("Our segments really do exist, awaiting handoff."); - } else { - log.warn("Our segments don't exist, giving up."); - return null; - } + log.warn("Our segments don't exist, giving up."); + return null; } } - - return segmentsAndMetadata; } - catch (InterruptedException e) { - throw e; - } - catch (Exception e) { - final long sleepMillis = computeNextRetrySleep(++nTry); - log.warn(e, "Failed publish (try %d), retrying in %,dms.", nTry, sleepMillis); - Thread.sleep(sleepMillis); + catch (IOException e) { + throw Throwables.propagate(e); } } - } + + return segmentsAndMetadata; + }, + publishExecutor ); } @@ -666,12 +657,4 @@ public void run() } }; } - - private static long computeNextRetrySleep(final long nTry) - { - final long baseSleepMillis = 1000; - final long maxSleepMillis = 60000; - final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * new Random().nextGaussian(), 0), 2); - return (long) (Math.min(maxSleepMillis, baseSleepMillis * Math.pow(2, nTry)) * fuzzyMultiplier); - } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 7b79ce421434..a4fe64f7f85f 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -48,6 +48,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; +import io.druid.java.util.common.RetryUtils; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -367,6 +368,9 @@ public ListenableFuture persist(Collection identifier int numPersistedRows = 0; for (SegmentIdentifier identifier : identifiers) { final Sink sink = sinks.get(identifier); + if (sink == null) { + throw new NullPointerException("No sink for identifier: " + identifier); + } final List hydrants = Lists.newArrayList(sink); commitHydrants.put(identifier, hydrants.size()); numPersistedRows += sink.getNumRowsInMemory(); @@ -590,9 +594,14 @@ private DataSegment mergeAndPush(final SegmentIdentifier identifier, final Sink tuningConfig.getIndexSpec() ); - DataSegment segment = dataSegmentPusher.push( - mergedFile, - sink.getSegment().withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes)) + // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types + final DataSegment segment = RetryUtils.retry( + () -> dataSegmentPusher.push( + mergedFile, + sink.getSegment().withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes)) + ), + exception -> exception instanceof Exception, + 5 ); objectMapper.writeValue(descriptorFile, segment); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java index 1f87e0d6f85b..af2aedecd446 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorDriverFailTest.java @@ -69,7 +69,7 @@ public class AppenderatorDriverFailTest { private static final String DATA_SOURCE = "foo"; private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - private static final long PUBLISH_TIMEOUT = 1000; + private static final long PUBLISH_TIMEOUT = 5000; private static final List ROWS = ImmutableList.of( new MapBasedInputRow( @@ -114,45 +114,15 @@ public void tearDown() throws Exception @Test public void testFailDuringPersist() throws IOException, InterruptedException, TimeoutException, ExecutionException - { - expectedException.expect(TimeoutException.class); - - driver = new AppenderatorDriver( - createPersistFailAppenderator(), - allocator, - segmentHandoffNotifierFactory, - new NoopUsedSegmentChecker(), - OBJECT_MAPPER, - new FireDepartmentMetrics() - ); - - driver.startJob(); - - final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); - segmentHandoffNotifierFactory.setHandoffDelay(100); - - Assert.assertNull(driver.startJob()); - - for (int i = 0; i < ROWS.size(); i++) { - committerSupplier.setMetadata(i + 1); - Assert.assertTrue(driver.add(ROWS.get(i), "dummy", committerSupplier).isOk()); - } - - driver.publish( - AppenderatorDriverTest.makeOkPublisher(), - committerSupplier.get(), - ImmutableList.of("dummy") - ).get(PUBLISH_TIMEOUT, TimeUnit.MILLISECONDS); - } - - @Test - public void testInterruptDuringPush() throws IOException, InterruptedException, TimeoutException, ExecutionException { expectedException.expect(ExecutionException.class); - expectedException.expectCause(CoreMatchers.instanceOf(InterruptedException.class)); + expectedException.expectCause(CoreMatchers.instanceOf(ISE.class)); + expectedException.expectMessage("Fail test while persisting segments" + + "[[foo_2000-01-01T00:00:00.000Z_2000-01-01T01:00:00.000Z_abc123, " + + "foo_2000-01-01T01:00:00.000Z_2000-01-01T02:00:00.000Z_abc123]]"); driver = new AppenderatorDriver( - createPushInterruptAppenderator(), + createPersistFailAppenderator(), allocator, segmentHandoffNotifierFactory, new NoopUsedSegmentChecker(), @@ -182,7 +152,11 @@ public void testInterruptDuringPush() throws IOException, InterruptedException, @Test public void testFailDuringPush() throws IOException, InterruptedException, TimeoutException, ExecutionException { - expectedException.expect(TimeoutException.class); + expectedException.expect(ExecutionException.class); + expectedException.expectCause(CoreMatchers.instanceOf(ISE.class)); + expectedException.expectMessage("Fail test while pushing segments" + + "[[foo_2000-01-01T00:00:00.000Z_2000-01-01T01:00:00.000Z_abc123, " + + "foo_2000-01-01T01:00:00.000Z_2000-01-01T02:00:00.000Z_abc123]]"); driver = new AppenderatorDriver( createPushFailAppenderator(), diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index a6baa240fe07..650069f47032 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -67,7 +67,7 @@ public class AppenderatorTest @Test public void testSimpleIngestion() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(2)) { + try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); boolean thrown; @@ -138,7 +138,7 @@ public SegmentIdentifier apply(DataSegment input) @Test public void testMaxRowsInMemory() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(3)) { + try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); final AtomicInteger eventCount = new AtomicInteger(0); final Supplier committerSupplier = new Supplier() @@ -191,7 +191,7 @@ public void run() public void testRestoreFromDisk() throws Exception { final RealtimeTuningConfig tuningConfig; - try (final AppenderatorTester tester = new AppenderatorTester(2)) { + try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); tuningConfig = tester.getTuningConfig(); @@ -233,7 +233,7 @@ public void run() appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 5), committerSupplier); appenderator.close(); - try (final AppenderatorTester tester2 = new AppenderatorTester(2, tuningConfig.getBasePersistDirectory())) { + try (final AppenderatorTester tester2 = new AppenderatorTester(2, tuningConfig.getBasePersistDirectory(), true)) { final Appenderator appenderator2 = tester2.getAppenderator(); Assert.assertEquals(ImmutableMap.of("eventCount", 4), appenderator2.startJob()); Assert.assertEquals(ImmutableList.of(IDENTIFIERS.get(0)), appenderator2.getSegments()); @@ -245,7 +245,7 @@ public void run() @Test(timeout = 10000L) public void testTotalRowCount() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(3)) { + try (final AppenderatorTester tester = new AppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); @@ -287,7 +287,7 @@ public void testTotalRowCount() throws Exception @Test public void testQueryByIntervals() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(2)) { + try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -423,7 +423,7 @@ public void testQueryByIntervals() throws Exception @Test public void testQueryBySegments() throws Exception { - try (final AppenderatorTester tester = new AppenderatorTester(2)) { + try (final AppenderatorTester tester = new AppenderatorTester(2, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 71fd3714d23c..c16bcf3beaf0 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -67,6 +67,7 @@ import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; public class AppenderatorTester implements AutoCloseable { @@ -89,12 +90,21 @@ public AppenderatorTester( final int maxRowsInMemory ) { - this(maxRowsInMemory, null); + this(maxRowsInMemory, null, false); } public AppenderatorTester( final int maxRowsInMemory, - final File basePersistDirectory + final boolean enablePushFailure + ) + { + this(maxRowsInMemory, null, enablePushFailure); + } + + public AppenderatorTester( + final int maxRowsInMemory, + final File basePersistDirectory, + final boolean enablePushFailure ) { objectMapper = new DefaultObjectMapper(); @@ -169,6 +179,8 @@ public int columnCacheSizeBytes() EmittingLogger.registerEmitter(emitter); dataSegmentPusher = new DataSegmentPusher() { + private boolean mustFail = true; + @Deprecated @Override public String getPathForHadoop(String dataSource) @@ -185,6 +197,12 @@ public String getPathForHadoop() @Override public DataSegment push(File file, DataSegment segment) throws IOException { + if (enablePushFailure && mustFail) { + mustFail = false; + throw new IOException("Push failure test"); + } else if (enablePushFailure) { + mustFail = true; + } pushedSegments.add(segment); return segment; } From 8ae9f62f803c680f635250e715456f1acff6bd41 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 30 Jun 2017 11:42:58 +0900 Subject: [PATCH 11/15] Fix doc --- .../druid/indexing/common/task/IndexTaskTest.java | 5 +++-- .../realtime/appenderator/Appenderator.java | 15 ++++++++++----- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 72177bbedcda..22e0437f4a4d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -37,9 +37,10 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.indexing.overlord.SegmentPublishResult; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.aggregation.AggregatorFactory; @@ -543,7 +544,7 @@ public void testWithSmallMaxTotalRows() throws Exception for (int i = 0; i < 6; i++) { final DataSegment segment = segments.get(i); - final Interval expectedInterval = new Interval(String.format("2014-01-01T0%d/PT1H", (i / 2))); + final Interval expectedInterval = new Interval(StringUtils.format("2014-01-01T0%d/PT1H", (i / 2))); final int expectedPartitionNum = i % 2; Assert.assertEquals("test", segment.getDataSource()); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java index 45caa34d9af4..0f808d9f2a0a 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -62,7 +62,8 @@ public interface Appenderator extends QuerySegmentWalker, Closeable * Committer is guaranteed to be *created* synchronously with the call to add, but will actually be used * asynchronously. *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread to keep the + * metadata committed by Committer in sync. * * @param identifier the segment into which this row should be added * @param row the row to add @@ -104,7 +105,8 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been * cleared. This may take some time, since all pending persists must finish first. * - * The add, clear, persist, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread to keep the + * metadata committed by Committer in sync. */ void clear() throws InterruptedException; @@ -128,7 +130,8 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * persist, but will actually be used asynchronously. Any metadata returned by the committer will be associated with * the data persisted to disk. *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread to keep the + * metadata committed by Committer in sync. * * @param identifiers segment identifiers to be persisted * @param committer a committer associated with all data that has been added to segments of the given identifiers so @@ -145,7 +148,8 @@ int add(SegmentIdentifier identifier, InputRow row, Supplier committe * be used asynchronously. Any metadata returned by the committer will be associated with the data persisted to * disk. *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread to keep the + * metadata committed by Committer in sync. * * @param committer a committer associated with all data that has been added so far * @@ -162,7 +166,8 @@ default ListenableFuture persistAll(Committer committer) *

* After this method is called, you cannot add new data to any segments that were previously under construction. *

- * The add, clear, persist, persistAll, and push methods should all be called from the same thread. + * The add, clear, persist, persistAll, and push methods should all be called from the same thread to keep the + * metadata committed by Committer in sync. * * @param identifiers list of segments to push * @param committer a committer associated with all data that has been added so far From f85f17d66ab01da030012e0e1366d079a9056a9f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 30 Jun 2017 12:04:17 +0900 Subject: [PATCH 12/15] Fix build failure --- .../druid/segment/realtime/appenderator/AppenderatorTester.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index c16bcf3beaf0..855ceb355e8e 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -67,7 +67,6 @@ import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadLocalRandom; public class AppenderatorTester implements AutoCloseable { From dc5e86f19be4d03455bae0fc61dec3a5befe79fb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 8 Jul 2017 10:57:34 +0900 Subject: [PATCH 13/15] Address comments --- .../appenderator/AppenderatorImpl.java | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 7ca8efcf8567..a2492e483f51 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -319,22 +319,20 @@ public void clear() throws InterruptedException // Drop commit metadata, then abandon all segments. try { - if (persistExecutor != null) { - final ListenableFuture uncommitFuture = persistExecutor.submit( - new Callable() + final ListenableFuture uncommitFuture = persistExecutor.submit( + new Callable() + { + @Override + public Object call() throws Exception { - @Override - public Object call() throws Exception - { - objectMapper.writeValue(computeCommitFile(), Committed.nil()); - return null; - } + objectMapper.writeValue(computeCommitFile(), Committed.nil()); + return null; } - ); + } + ); - // Await uncommit. - uncommitFuture.get(); - } + // Await uncommit. + uncommitFuture.get(); // Drop everything. final List> futures = Lists.newArrayList(); @@ -370,7 +368,7 @@ public ListenableFuture persist(Collection identifier for (SegmentIdentifier identifier : identifiers) { final Sink sink = sinks.get(identifier); if (sink == null) { - throw new NullPointerException("No sink for identifier: " + identifier); + throw new ISE("No sink for identifier: %s", identifier); } final List hydrants = Lists.newArrayList(sink); commitHydrants.put(identifier, hydrants.size()); @@ -472,7 +470,7 @@ public ListenableFuture push( for (final SegmentIdentifier identifier : identifiers) { final Sink sink = sinks.get(identifier); if (sink == null) { - throw new NullPointerException("No sink for identifier: " + identifier); + throw new ISE("No sink for identifier: %s", identifier); } theSinks.put(identifier, sink); sink.finishWriting(); From 248f52414397f0c2546f6ed38c50d8ff94f73906 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 8 Jul 2017 11:18:58 +0900 Subject: [PATCH 14/15] Fix compilation failure --- .../druid/indexing/common/task/IndexTaskTest.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 106b41741eec..f127434991b7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -804,11 +804,8 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception 0 ), null, - 2, - null, - false, - false, - true // report parse exception + createTuningConfig(2, null, null, null, false, false, true), // report parse exception + false ); IndexTask indexTask = new IndexTask( @@ -874,11 +871,8 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception 0 ), null, - 2, - null, - false, - false, - true // report parse exception + createTuningConfig(2, null, null, null, false, false, true), // report parse exception + false ); IndexTask indexTask = new IndexTask( From 1a3e8560893fe5bae301f95e55cea99ac78b1239 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 11 Jul 2017 08:38:49 +0900 Subject: [PATCH 15/15] Fix transient test failure --- .../java/io/druid/indexing/common/task/IndexTaskTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index f127434991b7..f248d5e93adf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -804,7 +804,7 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception 0 ), null, - createTuningConfig(2, null, null, null, false, false, true), // report parse exception + createTuningConfig(2, 1, null, null, false, true, true), // report parse exception false ); @@ -822,6 +822,10 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception Assert.assertEquals(2, segments.size()); Assert.assertNotEquals(segments.get(0), segments.get(1)); + for (DataSegment segment : segments) { + System.out.println(segment.getDimensions()); + } + for (int i = 0; i < 2; i++) { final DataSegment segment = segments.get(i); final Set dimensions = new HashSet<>(segment.getDimensions());