From 8ae97fd33aaa4c1e3800881c80650aa6b61b4dda Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 20 Oct 2017 11:04:38 +0900 Subject: [PATCH 01/19] Add compaction task --- docs/content/ingestion/firehose.md | 2 +- .../indexing/common/task/AbstractTask.java | 39 ++- .../indexing/common/task/ArchiveTask.java | 2 +- .../indexing/common/task/CompactionTask.java | 255 ++++++++++++++++++ .../common/task/CompactionTaskUtils.java | 81 ++++++ .../common/task/HadoopConverterTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 110 +++++--- .../druid/indexing/common/task/KillTask.java | 2 +- .../druid/indexing/common/task/MoveTask.java | 2 +- .../indexing/common/task/RestoreTask.java | 2 +- .../io/druid/indexing/common/task/Task.java | 3 +- .../IngestSegmentFirehoseFactory.java | 132 ++++----- .../io/druid/indexing/common/TestUtils.java | 2 +- .../IngestSegmentFirehoseFactoryTest.java | 4 +- .../CoordinatorResourceTestClient.java | 3 +- .../tests/indexer/AbstractIndexerTest.java | 4 +- .../tests/indexer/ITCompactionTaskTest.java | 87 ++++++ .../test/resources/indexer/union_queries.json | 2 +- .../indexer/wikipedia_compaction_task.json | 5 + .../indexer/wikipedia_index_data.json | 7 +- .../indexer/wikipedia_index_queries.json | 16 +- .../indexer/wikipedia_index_task.json | 2 +- 22 files changed, 610 insertions(+), 154 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/ITCompactionTaskTest.java create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_compaction_task.json diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md index aec3ea15f62d..e1c2e73931e2 100644 --- a/docs/content/ingestion/firehose.md +++ b/docs/content/ingestion/firehose.md @@ -79,7 +79,7 @@ A sample ingest firehose spec is shown below - |interval|A String representing ISO-8601 Interval. This defines the time range to fetch the data over.|yes| |dimensions|The list of dimensions to select. If left empty, no dimensions are returned. If left null or not defined, all dimensions are returned. |no| |metrics|The list of metrics to select. If left empty, no metrics are returned. If left null or not defined, all metrics are selected.|no| -|filter| See [Filters](../querying/filters.html)|yes| +|filter| See [Filters](../querying/filters.html)|no| #### CombiningFirehose diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java index 9fd6fa64c557..b18be664eb44 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java @@ -33,7 +33,9 @@ import io.druid.query.QueryRunner; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -80,15 +82,27 @@ protected AbstractTask( this.context = context; } - public static String makeId(String id, final String typeName, String dataSource, Interval interval) + static String getOrMakeId(String id, final String typeName, String dataSource) { - return id != null ? id : joinId( - typeName, - dataSource, - interval.getStart(), - interval.getEnd(), - DateTimes.nowUtc().toString() - ); + return getOrMakeId(id, typeName, dataSource, null); + } + + static String getOrMakeId(String id, final String typeName, String dataSource, @Nullable Interval interval) + { + if (id != null) { + return id; + } + + final List objects = new ArrayList<>(); + objects.add(typeName); + objects.add(dataSource); + if (interval != null) { + objects.add(interval.getStart()); + objects.add(interval.getEnd()); + } + objects.add(DateTimes.nowUtc().toString()); + + return joinId(objects); } @JsonProperty @@ -167,7 +181,12 @@ public String toString() * * @return string of joined objects */ - public static String joinId(Object... objects) + static String joinId(List objects) + { + return ID_JOINER.join(objects); + } + + static String joinId(Object...objects) { return ID_JOINER.join(objects); } @@ -202,7 +221,7 @@ public int hashCode() return id.hashCode(); } - protected List getTaskLocks(TaskActionClient client) throws IOException + static List getTaskLocks(TaskActionClient client) throws IOException { return client.submit(new LockListAction()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java index 0a45a4f27da8..32787755a8f1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java @@ -47,7 +47,7 @@ public ArchiveTask( ) { super( - makeId(id, "archive", dataSource, interval), + getOrMakeId(id, "archive", dataSource, interval), dataSource, interval, context diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java new file mode 100644 index 000000000000..a702a00236a3 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -0,0 +1,255 @@ +/* + * 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.indexing.common.task; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Injector; +import io.druid.data.input.impl.NoopInputRowParser; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.IndexTask.IndexIOConfig; +import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.granularity.Granularity; +import io.druid.java.util.common.granularity.PeriodGranularity; +import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.IndexIO; +import io.druid.segment.QueryableIndex; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import org.joda.time.Interval; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; + +public class CompactionTask extends AbstractTask +{ + private static final Logger log = new Logger(CompactionTask.class); + private static final String TYPE = "compact"; + + private final Interval interval; + private final IndexTuningConfig tuningConfig; + private final Injector injector; + private final IndexIO indexIO; + private final ObjectMapper jsonMapper; + + @JsonIgnore + private IndexTask indexTaskSpec; + + @JsonCreator + public CompactionTask( + @JsonProperty("id") final String id, + @JsonProperty("resource") final TaskResource taskResource, + @JsonProperty("dataSource") final String dataSource, + @JsonProperty("interval") final Interval interval, + @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, + @JsonProperty("context") final Map context, + @JacksonInject Injector injector, + @JacksonInject IndexIO indexIO, + @JacksonInject ObjectMapper jsonMapper + ) + { + super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); + this.interval = Preconditions.checkNotNull(interval, "interval"); + this.tuningConfig = tuningConfig; + this.injector = injector; + this.indexIO = indexIO; + this.jsonMapper = jsonMapper; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public IndexTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_MERGE_TASK_PRIORITY); + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + intervals.add(interval); + return IndexTask.isReady(taskActionClient, intervals); + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + if (indexTaskSpec == null) { + final IndexIngestionSpec ingestionSpec = createIngestionSchema( + toolbox, + getDataSource(), + interval, + tuningConfig, + indexIO, + injector, + jsonMapper + ); + + indexTaskSpec = new IndexTask( + getId(), + getGroupId(), + getTaskResource(), + getDataSource(), + ingestionSpec, + getContext() + ); + } + + final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(indexTaskSpec); + log.info("Generated compaction task details: " + json); + + return indexTaskSpec.run(toolbox); + } + + private static IndexIngestionSpec createIngestionSchema( + TaskToolbox toolbox, + String dataSource, + Interval interval, + IndexTuningConfig tuningConfig, + IndexIO indexIO, + Injector injector, + ObjectMapper jsonMapper + ) throws IOException, SegmentLoadingException + { + Pair, List>> pair = prepareSegments( + toolbox, + dataSource, + interval + ); + final Map segmentFileMap = pair.lhs; + final List> timelineSegments = pair.rhs; + final List dimensions = IngestSegmentFirehoseFactory.getUniqueDimensions( + timelineSegments, + new NoopInputRowParser(null) + ); + final List metrics = IngestSegmentFirehoseFactory.getUniqueMetrics(timelineSegments); + return new IndexIngestionSpec( + createDataSchema(dataSource, interval, indexIO, jsonMapper, timelineSegments, segmentFileMap), + new IndexIOConfig( + new IngestSegmentFirehoseFactory(dataSource, interval, null, dimensions, metrics, injector, indexIO), + false + ), + tuningConfig + ); + } + + private static Pair, List>> prepareSegments( + TaskToolbox toolbox, + String dataSource, + Interval interval + ) throws IOException, SegmentLoadingException + { + final List usedSegments = toolbox + .getTaskActionClient() + .submit(new SegmentListUsedAction(dataSource, interval, null)); + final Map segmentFileMap = toolbox.fetchSegments(usedSegments); + final List> timelineSegments = CompactionTaskUtils.toTimelineSegments( + usedSegments, + interval + ); + return Pair.of(segmentFileMap, timelineSegments); + } + + private static DataSchema createDataSchema( + String dataSource, + Interval interval, + IndexIO indexIO, + ObjectMapper jsonMapper, + List> timelineSegments, + Map segmentFileMap + ) + throws IOException, SegmentLoadingException + { + // find metadata for interval + final List segments = CompactionTaskUtils.loadSegments(timelineSegments, segmentFileMap, indexIO); + final Granularity expectedGranularity = segments.get(0).getMetadata().getQueryGranularity(); + final boolean expectedRollup = segments.get(0).getMetadata().isRollup(); + + // check metadata + Preconditions.checkState( + segments.stream().allMatch(index -> index.getMetadata().getQueryGranularity().equals(expectedGranularity)) + ); + Preconditions.checkState( + segments.stream().allMatch(index -> index.getMetadata().isRollup() == expectedRollup) + ); + + // find merged aggregators + final List aggregatorFactories = segments.stream() + .map(index -> index.getMetadata().getAggregators()) + .collect(Collectors.toList()); + final AggregatorFactory[] mergedAggregators = AggregatorFactory.mergeAggregators(aggregatorFactories); + + // find granularity spec + final GranularitySpec granularitySpec = new UniformGranularitySpec( + new PeriodGranularity(interval.toPeriod(), null, null), + expectedGranularity, + expectedRollup, + ImmutableList.of(interval) + ); + + return new DataSchema( + dataSource, + ImmutableMap.of("type", "noop"), + mergedAggregators, + granularitySpec, + jsonMapper + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java new file mode 100644 index 000000000000..8ef60ffb2b51 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.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.indexing.common.task; + +import com.google.common.base.Preconditions; +import io.druid.java.util.common.guava.Comparators; +import io.druid.segment.IndexIO; +import io.druid.segment.QueryableIndex; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.PartitionHolder; +import org.joda.time.Interval; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class CompactionTaskUtils +{ + public static List> toTimelineSegments( + List usedSegments, + Interval interval + ) throws IOException, SegmentLoadingException + { + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( + Comparators.naturalNullsFirst() + ); + + for (DataSegment segment : usedSegments) { + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + } + return timeline.lookup(interval); + } + + public static List loadSegments( + List> timelineSegments, + Map segmentFileMap, + IndexIO indexIO + ) throws IOException + { + final List segments = new ArrayList<>(); + + for (TimelineObjectHolder timelineSegment : timelineSegments) { + final PartitionHolder partitionHolder = timelineSegment.getObject(); + for (PartitionChunk chunk : partitionHolder) { + final DataSegment segment = chunk.getObject(); + segments.add( + indexIO.loadIndex( + Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier()) + ) + ); + } + } + + return segments; + } + + private CompactionTaskUtils() {} +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java index a83515a6efe1..3ca5097f7674 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java @@ -69,7 +69,7 @@ public HadoopConverterTask( ) { super( - makeId( + getOrMakeId( id, TYPE, Preconditions.checkNotNull(dataSource, "dataSource"), 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 d4be01831cce..70570e2d8871 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 @@ -36,8 +36,6 @@ import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ListenableFuture; -import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.JodaUtils; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -53,6 +51,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.JodaUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Comparators; @@ -115,31 +114,6 @@ public class IndexTask extends AbstractTask private static final HashFunction hashFunction = Hashing.murmur3_128(); private static final String TYPE = "index"; - private static String makeId(String id, IndexIngestionSpec ingestionSchema) - { - if (id != null) { - return id; - } else { - return StringUtils.format("index_%s_%s", makeDataSource(ingestionSchema), DateTimes.nowUtc()); - } - } - - private static String makeGroupId(IndexIngestionSpec ingestionSchema) - { - if (ingestionSchema.getIOConfig().isAppendToExisting()) { - // Shared locking group for all tasks that append, since they are OK to run concurrently. - return StringUtils.format("%s_append_%s", TYPE, ingestionSchema.getDataSchema().getDataSource()); - } else { - // Return null, one locking group per task. - return null; - } - } - - private static String makeDataSource(IndexIngestionSpec ingestionSchema) - { - return ingestionSchema.getDataSchema().getDataSource(); - } - @JsonIgnore private final IndexIngestionSpec ingestionSchema; @@ -151,17 +125,66 @@ public IndexTask( @JsonProperty("context") final Map context ) { - super( - makeId(id, ingestionSchema), - makeGroupId(ingestionSchema), + this( + id, taskResource, - makeDataSource(ingestionSchema), + ingestionSchema.dataSchema.getDataSource(), + ingestionSchema.ioConfig.appendToExisting, + ingestionSchema, + context + ); + } + + IndexTask( + String id, + TaskResource resource, + String dataSource, + boolean appendToExisting, + IndexIngestionSpec ingestionSchema, + Map context + ) + { + this( + getOrMakeId(id, TYPE, dataSource), + makeGroupId(appendToExisting, dataSource), + resource, + dataSource, + ingestionSchema, + context + ); + } + + IndexTask( + String id, + String groupId, + TaskResource resource, + String dataSource, + IndexIngestionSpec ingestionSchema, + Map context + ) + { + super( + getOrMakeId(id, TYPE, dataSource), + groupId, + resource, + dataSource, context ); this.ingestionSchema = ingestionSchema; } + private static String makeGroupId(boolean isAppendToExisting, String dataSource) + { + if (isAppendToExisting) { + // Shared locking group for all tasks that append, since they are OK to run concurrently. + return StringUtils.format("%s_append_%s", TYPE, dataSource); + } else { + // Return null, one locking group per task. + return null; + } + } + @Override public int getPriority() { @@ -182,21 +205,26 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception .bucketIntervals(); if (intervals.isPresent()) { - final List locks = getTaskLocks(taskActionClient); - if (locks.size() == 0) { - try { - Tasks.tryAcquireExclusiveLocks(taskActionClient, intervals.get()); - } - catch (Exception e) { - return false; - } - } - return true; + return isReady(taskActionClient, intervals.get()); } else { return true; } } + static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException + { + final List locks = getTaskLocks(actionClient); + if (locks.size() == 0) { + try { + Tasks.tryAcquireExclusiveLocks(actionClient, intervals); + } + catch (Exception e) { + return false; + } + } + return true; + } + @JsonProperty("spec") public IndexIngestionSpec getIngestionSchema() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java index 6873f665f128..3f5d48f2c909 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java @@ -52,7 +52,7 @@ public KillTask( ) { super( - makeId(id, "kill", dataSource, interval), + getOrMakeId(id, "kill", dataSource, interval), dataSource, interval, context diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index 49454d99ac41..6bafa61254d7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -55,7 +55,7 @@ public MoveTask( ) { super( - makeId(id, "move", dataSource, interval), + getOrMakeId(id, "move", dataSource, interval), dataSource, interval, context diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java index 680e50954cf7..62e15edab3ce 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RestoreTask.java @@ -48,7 +48,7 @@ public RestoreTask( ) { super( - makeId(id, "restore", dataSource, interval), + getOrMakeId(id, "restore", dataSource, interval), dataSource, interval, context diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 21297f7d6a10..8e18d725e436 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -60,7 +60,8 @@ @JsonSubTypes.Type(name = "version_converter_sub", value = ConvertSegmentBackwardsCompatibleTask.SubTask.class), // backwards compat - Deprecated @JsonSubTypes.Type(name = "convert_segment", value = ConvertSegmentTask.class), @JsonSubTypes.Type(name = "convert_segment_sub", value = ConvertSegmentTask.SubTask.class), - @JsonSubTypes.Type(name = "same_interval_merge", value = SameIntervalMergeTask.class) + @JsonSubTypes.Type(name = "same_interval_merge", value = SameIntervalMergeTask.class), + @JsonSubTypes.Type(name = "compact", value = CompactionTask.class) }) public interface Task { 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 231c8b19ebdc..2277d7097ce3 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 @@ -36,8 +36,8 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.task.CompactionTaskUtils; import io.druid.indexing.common.task.NoopTask; -import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.filter.DimFilter; import io.druid.segment.IndexIO; @@ -47,15 +47,17 @@ import io.druid.segment.realtime.firehose.WindowedStorageAdapter; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; -import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; public class IngestSegmentFirehoseFactory implements FirehoseFactory { @@ -143,14 +145,8 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) .getTaskActionClient() .submit(new SegmentListUsedAction(dataSource, interval, null)); final Map segmentFileMap = taskToolbox.fetchSegments(usedSegments); - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - Comparators.naturalNullsFirst() - ); - - for (DataSegment segment : usedSegments) { - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - } - final List> timeLineSegments = timeline.lookup( + final List> timeLineSegments = CompactionTaskUtils.toTimelineSegments( + usedSegments, interval ); @@ -160,84 +156,16 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) } else if (inputRowParser.getParseSpec().getDimensionsSpec().hasCustomDimensions()) { dims = inputRowParser.getParseSpec().getDimensionsSpec().getDimensionNames(); } else { - Set dimSet = Sets.newHashSet( - Iterables.concat( - Iterables.transform( - timeLineSegments, - new Function, Iterable>() - { - @Override - public Iterable apply( - TimelineObjectHolder timelineObjectHolder - ) - { - return Iterables.concat( - Iterables.transform( - timelineObjectHolder.getObject(), - new Function, Iterable>() - { - @Override - public Iterable apply(PartitionChunk input) - { - return input.getObject().getDimensions(); - } - } - ) - ); - } - } - - ) - ) - ); - dims = Lists.newArrayList( - Sets.difference( - dimSet, - inputRowParser - .getParseSpec() - .getDimensionsSpec() - .getDimensionExclusions() - ) - ); + dims = getUniqueDimensions(timeLineSegments, inputRowParser); } final List metricsList; if (metrics != null) { metricsList = metrics; } else { - Set metricsSet = Sets.newHashSet( - Iterables.concat( - Iterables.transform( - timeLineSegments, - new Function, Iterable>() - { - @Override - public Iterable apply( - TimelineObjectHolder input - ) - { - return Iterables.concat( - Iterables.transform( - input.getObject(), - new Function, Iterable>() - { - @Override - public Iterable apply(PartitionChunk input) - { - return input.getObject().getMetrics(); - } - } - ) - ); - } - } - ) - ) - ); - metricsList = Lists.newArrayList(metricsSet); + metricsList = getUniqueMetrics(timeLineSegments); } - final List adapters = Lists.newArrayList( Iterables.concat( Iterables.transform( @@ -286,6 +214,50 @@ public WindowedStorageAdapter apply(final PartitionChunk input) catch (IOException | SegmentLoadingException e) { throw Throwables.propagate(e); } + } + + public static List getUniqueDimensions( + List> timelineSegments, + InputRowParser inputRowParser + ) + { + final Set dimSet = timelineSegments + .stream() + .flatMap(timelineHolder -> { + final PartitionHolder partitionHolder = timelineHolder.getObject(); + final List dims = new ArrayList<>(partitionHolder.size()); + for (PartitionChunk chunk : partitionHolder) { + dims.addAll(chunk.getObject().getDimensions()); + } + return dims.stream(); + }) + .collect(Collectors.toSet()); + + return new ArrayList<>( + Sets.difference( + dimSet, + inputRowParser + .getParseSpec() + .getDimensionsSpec() + .getDimensionExclusions() + ) + ); + } + + public static List getUniqueMetrics(List> timelineSegments) + { + final Set metricsSet = timelineSegments + .stream() + .flatMap(timelineHolder -> { + final PartitionHolder partitionHolder = timelineHolder.getObject(); + final List metrics = new ArrayList<>(partitionHolder.size()); + for (PartitionChunk chunk : partitionHolder) { + metrics.addAll(chunk.getObject().getMetrics()); + } + return metrics.stream(); + }) + .collect(Collectors.toSet()); + return new ArrayList<>(metricsSet); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index c7bbb95471b9..f655f57225c9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -53,7 +53,7 @@ public class TestUtils public TestUtils() { - jsonMapper = new DefaultObjectMapper(); + this.jsonMapper = new DefaultObjectMapper(); indexIO = new IndexIO( jsonMapper, new ColumnConfig() 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 f935e94be147..bbdf264741e9 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 @@ -33,8 +33,6 @@ import com.google.inject.Guice; import com.google.inject.Module; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; @@ -56,6 +54,8 @@ import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.java.util.common.IOE; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; diff --git a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java index e4231d3e2219..f5c05db1b325 100644 --- a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java +++ b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java @@ -38,6 +38,7 @@ import java.net.URL; import java.util.ArrayList; +import java.util.List; import java.util.Map; public class CoordinatorResourceTestClient @@ -79,7 +80,7 @@ private String getLoadStatusURL() } // return a list of the segment dates for the specified datasource - public ArrayList getSegmentIntervals(final String dataSource) throws Exception + public List getSegmentIntervals(final String dataSource) throws Exception { ArrayList segments = null; try { diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java index 148482cd88bf..18539f00a972 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java @@ -33,8 +33,8 @@ import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.concurrent.Callable; public abstract class AbstractIndexerTest @@ -55,7 +55,7 @@ public abstract class AbstractIndexerTest protected void unloadAndKillData(final String dataSource) throws Exception { - ArrayList intervals = coordinator.getSegmentIntervals(dataSource); + List intervals = coordinator.getSegmentIntervals(dataSource); // each element in intervals has this form: // 2015-12-01T23:15:00.000Z/2015-12-01T23:16:00.000Z diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITCompactionTaskTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITCompactionTaskTest.java new file mode 100644 index 000000000000..e82d0155d4ec --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITCompactionTaskTest.java @@ -0,0 +1,87 @@ +/* + * 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.tests.indexer; + +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import io.druid.testing.guice.DruidTestModuleFactory; +import io.druid.testing.utils.RetryUtil; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.List; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITCompactionTaskTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(ITCompactionTaskTest.class); + private static String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static String INDEX_DATASOURCE = "wikipedia_index_test"; + private static String COMPACTION_TASK = "/indexer/wikipedia_compaction_task.json"; + private static String COMPACTED_INTERVAL = "2013-08-31T00:00:00.000Z/2013-09-02T00:00:00.000Z"; + + @Test + public void testCompaction() throws Exception + { + loadData(); + final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(INDEX_DATASOURCE); + if (intervalsBeforeCompaction.contains(COMPACTED_INTERVAL)) { + throw new ISE("Containing a segment for the compacted interval[%s] before compaction", COMPACTED_INTERVAL); + } + try { + queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); + compactData(); + queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); + + final List intervalsAfterCompaction = coordinator.getSegmentIntervals(INDEX_DATASOURCE); + if (!intervalsAfterCompaction.contains(COMPACTED_INTERVAL)) { + throw new ISE("Compacted segment for interval[%s] does not exist", COMPACTED_INTERVAL); + } + } + finally { + unloadAndKillData(INDEX_DATASOURCE); + } + } + + private void loadData() throws Exception + { + final String taskID = indexer.submitTask(getTaskAsString(INDEX_TASK)); + LOG.info("TaskID for loading index task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + RetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE), + "Segment Load" + ); + } + + private void compactData() throws Exception + { + final String taskID = indexer.submitTask(getTaskAsString(COMPACTION_TASK)); + LOG.info("TaskID for compaction task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + RetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE), + "Segment Compaction" + ); + } +} diff --git a/integration-tests/src/test/resources/indexer/union_queries.json b/integration-tests/src/test/resources/indexer/union_queries.json index ec13d507d42f..fa63e8404d49 100644 --- a/integration-tests/src/test/resources/indexer/union_queries.json +++ b/integration-tests/src/test/resources/indexer/union_queries.json @@ -558,7 +558,7 @@ "timestamp": "2013-08-31T01:02:33.000Z", "result": { "minTime": "2013-08-31T01:02:33.000Z", - "maxTime": "2013-08-31T12:41:27.000Z" + "maxTime": "2013-09-01T12:41:27.000Z" } } ] diff --git a/integration-tests/src/test/resources/indexer/wikipedia_compaction_task.json b/integration-tests/src/test/resources/indexer/wikipedia_compaction_task.json new file mode 100644 index 000000000000..bc7f786646fb --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_compaction_task.json @@ -0,0 +1,5 @@ +{ + "type" : "compact", + "dataSource" : "wikipedia_index_test", + "interval" : "2013-08-31/2013-09-02" +} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_data.json b/integration-tests/src/test/resources/indexer/wikipedia_index_data.json index 592996e18056..b186657dbcf0 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_data.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_data.json @@ -2,4 +2,9 @@ {"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} {"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} {"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} -{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json index eb30d794e65c..873ed26c43a7 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_queries.json @@ -5,13 +5,15 @@ "queryType" : "timeBoundary", "dataSource": "wikipedia_index_test" }, - "expectedResults":[ { - "timestamp" : "2013-08-31T01:02:33.000Z", - "result" : { - "minTime" : "2013-08-31T01:02:33.000Z", - "maxTime" : "2013-08-31T12:41:27.000Z" - } - } ] + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] }, { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json index 369f9aac6b44..b0b6596c95ac 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json @@ -27,7 +27,7 @@ "granularitySpec": { "segmentGranularity": "DAY", "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-01" ] + "intervals" : [ "2013-08-31/2013-09-02" ] }, "parser": { "parseSpec": { From 26dad052ba740fc6159ad1d8d2f76c92abc4f2f6 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 20 Oct 2017 12:24:52 +0900 Subject: [PATCH 02/19] added doc --- docs/content/ingestion/tasks.md | 84 ++++++++++++++++++++++++++------- 1 file changed, 66 insertions(+), 18 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index be39454b5244..fe4b7d2e81e9 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -104,7 +104,7 @@ Tasks can have different default priorities depening on their types. Here are a |---------|----------------| |Realtime index task|75| |Batch index task|50| -|Merge/Append task|25| +|Merge/Append/Compation task|25| |Other tasks|0| You can override the task priority by setting your priority in the task context like below. @@ -184,19 +184,6 @@ On the contrary, in the incremental publishing mode, segments are incrementally 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. -### Task Context - -The task context is used for various task configuration parameters. The following parameters apply to all tasks. - -|property|default|description| -|--------|-------|-----------| -|taskLockTimeout|300000|task lock timeout in millisecond. For more details, see [the below Locking section](#locking).| - -
-When a task acquires a lock, it sends a request via HTTP and awaits until it receives a response containing the lock acquisition result. -As a result, an HTTP timeout error can occur if `taskLockTimeout` is greater than `druid.server.http.maxIdleTime` of overlords. -
- Segment Merging Tasks --------------------- @@ -210,7 +197,8 @@ Append tasks append a list of segments together into a single segment (one after "id": , "dataSource": , "segments": , - "aggregations": + "aggregations": , + "context": } ``` @@ -228,7 +216,8 @@ The grammar is: "dataSource": , "aggregations": , "rollup": , - "segments": + "segments": , + "context": } ``` @@ -245,10 +234,53 @@ The grammar is: "dataSource": , "aggregations": , "rollup": , - "interval": + "interval": , + "context": } ``` +### Compaction Task + +Compaction tasks merge all segments of the given interval. The syntax is: + +```json +{ + "type": "compact", + "id": , + "dataSource": , + "interval": , + "tuningConfig" , + "context": +} +``` + +|Field|Description|Required| +|-----|-----------|--------| +|`type`|Task type. Should be `compact`|Yes| +|`id`|Task id|No| +|`dataSource`|dataSource name to be compacted|Yes| +|`interval`|interval of segments to be compacted|Yes| +|`tuningConfig`|[Index task tuningConfig](#tuningconfig)|No| +|`context`|[Task context](#taskcontext)|No| + +An example of compaction task is + +```json +{ + "type" : "compact", + "dataSource" : "wikipedia", + "interval" : "2017-01-01/2018-01-01" +} +``` + +This compaction task merges _all segments_ of the interval `2017-01-01/2018-01-01`. + +A compaction task internally generates an indexTask spec for performing compaction work with some fixed parameters. +For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html) and `dimensionsSpec` and `metricsSpec` +always include all dimensions and metrics of the input segments. + +Note that all input segments should have the same `queryGranularity` and `rollup`. See [Segment Metadata Queries](../querying/segmentmetadataquery.html#analysistypes) for more details. + Segment Destroying Tasks ------------------------ @@ -261,7 +293,8 @@ Kill tasks delete all information about a segment and removes it from deep stora "type": "kill", "id": , "dataSource": , - "interval" : + "interval" : , + "context": } ``` @@ -342,6 +375,21 @@ These tasks start, sleep for a time and are used only for testing. The available } ``` +Task Context +------------ + +The task context is used for various task configuration parameters. The following parameters apply to all task types. + +|property|default|description| +|--------|-------|-----------| +|taskLockTimeout|300000|task lock timeout in millisecond. For more details, see [the below Locking section](#locking).| +|priority|Different based on task types. See [Task Priority](#task-priority).|Task priority| + +
+When a task acquires a lock, it sends a request via HTTP and awaits until it receives a response containing the lock acquisition result. +As a result, an HTTP timeout error can occur if `taskLockTimeout` is greater than `druid.server.http.maxIdleTime` of overlords. +
+ Locking ------- From 9aa7fff1590d0c9e32ab44d583ae350ef9962597 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Oct 2017 08:46:34 +0900 Subject: [PATCH 03/19] use combining aggregators --- .../druid/indexing/common/task/CompactionTask.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index a702a00236a3..fbab012196d1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -55,6 +55,7 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.SortedSet; @@ -235,6 +236,16 @@ private static DataSchema createDataSchema( .map(index -> index.getMetadata().getAggregators()) .collect(Collectors.toList()); final AggregatorFactory[] mergedAggregators = AggregatorFactory.mergeAggregators(aggregatorFactories); + // conver to combining aggregators + final AggregatorFactory[] combiningAggregators; + if (mergedAggregators == null) { + combiningAggregators = null; + } else { + final List combiningAggregatorList = Arrays.stream(mergedAggregators) + .map(AggregatorFactory::getCombiningFactory) + .collect(Collectors.toList()); + combiningAggregators = combiningAggregatorList.toArray(new AggregatorFactory[combiningAggregatorList.size()]); + } // find granularity spec final GranularitySpec granularitySpec = new UniformGranularitySpec( @@ -247,7 +258,7 @@ private static DataSchema createDataSchema( return new DataSchema( dataSource, ImmutableMap.of("type", "noop"), - mergedAggregators, + combiningAggregators, granularitySpec, jsonMapper ); From 25c7b00cee1e0ef0741b02d6dcc1eb23d330b669 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 25 Oct 2017 16:19:47 +0900 Subject: [PATCH 04/19] address comments --- docs/content/ingestion/tasks.md | 9 +- .../indexing/common/task/CompactionTask.java | 94 ++++++++++++------- .../common/task/CompactionTaskUtils.java | 81 ---------------- .../druid/indexing/common/task/IndexTask.java | 26 ++--- .../IngestSegmentFirehoseFactory.java | 13 ++- 5 files changed, 75 insertions(+), 148 deletions(-) delete mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index fe4b7d2e81e9..fe2237cd397e 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -104,7 +104,7 @@ Tasks can have different default priorities depening on their types. Here are a |---------|----------------| |Realtime index task|75| |Batch index task|50| -|Merge/Append/Compation task|25| +|Merge/Append/Compaction task|25| |Other tasks|0| You can override the task priority by setting your priority in the task context like below. @@ -273,13 +273,14 @@ An example of compaction task is } ``` -This compaction task merges _all segments_ of the interval `2017-01-01/2018-01-01`. +This compaction task merges _all segments_ of the interval `2017-01-01/2018-01-01` into a _single segment_. -A compaction task internally generates an indexTask spec for performing compaction work with some fixed parameters. +A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters. For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html) and `dimensionsSpec` and `metricsSpec` always include all dimensions and metrics of the input segments. -Note that all input segments should have the same `queryGranularity` and `rollup`. See [Segment Metadata Queries](../querying/segmentmetadataquery.html#analysistypes) for more details. +Note that the output segment is rolled up only when `rollup` is set for all input segments. +See [Segment Metadata Queries](../querying/segmentmetadataquery.html#analysistypes) for more details about `rollup`. Segment Destroying Tasks ------------------------ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index fbab012196d1..9637ec558d59 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -28,7 +28,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; -import io.druid.data.input.impl.NoopInputRowParser; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; @@ -37,24 +36,28 @@ import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; 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.granularity.PeriodGranularity; +import io.druid.java.util.common.granularity.NoneGranularity; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec; -import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -176,15 +179,18 @@ private static IndexIngestionSpec createIngestionSchema( ); final Map segmentFileMap = pair.lhs; final List> timelineSegments = pair.rhs; - final List dimensions = IngestSegmentFirehoseFactory.getUniqueDimensions( - timelineSegments, - new NoopInputRowParser(null) - ); - final List metrics = IngestSegmentFirehoseFactory.getUniqueMetrics(timelineSegments); return new IndexIngestionSpec( createDataSchema(dataSource, interval, indexIO, jsonMapper, timelineSegments, segmentFileMap), new IndexIOConfig( - new IngestSegmentFirehoseFactory(dataSource, interval, null, dimensions, metrics, injector, indexIO), + new IngestSegmentFirehoseFactory( + dataSource, + interval, + null, // no filter + null, // null means all unique dimensions + null, // null means all unique metrics + injector, + indexIO + ), false ), tuningConfig @@ -201,10 +207,9 @@ private static Pair, List segmentFileMap = toolbox.fetchSegments(usedSegments); - final List> timelineSegments = CompactionTaskUtils.toTimelineSegments( - usedSegments, - interval - ); + final List> timelineSegments = VersionedIntervalTimeline + .forSegments(usedSegments) + .lookup(interval); return Pair.of(segmentFileMap, timelineSegments); } @@ -219,39 +224,33 @@ private static DataSchema createDataSchema( throws IOException, SegmentLoadingException { // find metadata for interval - final List segments = CompactionTaskUtils.loadSegments(timelineSegments, segmentFileMap, indexIO); - final Granularity expectedGranularity = segments.get(0).getMetadata().getQueryGranularity(); - final boolean expectedRollup = segments.get(0).getMetadata().isRollup(); + final List segments = loadSegments(timelineSegments, segmentFileMap, indexIO); - // check metadata - Preconditions.checkState( - segments.stream().allMatch(index -> index.getMetadata().getQueryGranularity().equals(expectedGranularity)) - ); - Preconditions.checkState( - segments.stream().allMatch(index -> index.getMetadata().isRollup() == expectedRollup) - ); + // set rollup only if rollup is set for all segments + final boolean rollup = segments.stream().allMatch(index -> index.getMetadata().isRollup()); // find merged aggregators final List aggregatorFactories = segments.stream() .map(index -> index.getMetadata().getAggregators()) .collect(Collectors.toList()); final AggregatorFactory[] mergedAggregators = AggregatorFactory.mergeAggregators(aggregatorFactories); - // conver to combining aggregators - final AggregatorFactory[] combiningAggregators; + if (mergedAggregators == null) { - combiningAggregators = null; - } else { - final List combiningAggregatorList = Arrays.stream(mergedAggregators) - .map(AggregatorFactory::getCombiningFactory) - .collect(Collectors.toList()); - combiningAggregators = combiningAggregatorList.toArray(new AggregatorFactory[combiningAggregatorList.size()]); + throw new ISE("Failed to merge aggregators[%s]", aggregatorFactories); } + // conver to combining aggregators + final List combiningAggregatorList = Arrays.stream(mergedAggregators) + .map(AggregatorFactory::getCombiningFactory) + .collect(Collectors.toList()); + final AggregatorFactory[] combiningAggregators = combiningAggregatorList.toArray( + new AggregatorFactory[combiningAggregatorList.size()] + ); + // find granularity spec - final GranularitySpec granularitySpec = new UniformGranularitySpec( - new PeriodGranularity(interval.toPeriod(), null, null), - expectedGranularity, - expectedRollup, + final GranularitySpec granularitySpec = new ArbitraryGranularitySpec( + new NoneGranularity(), + rollup, ImmutableList.of(interval) ); @@ -263,4 +262,27 @@ private static DataSchema createDataSchema( jsonMapper ); } + + private static List loadSegments( + List> timelineSegments, + Map segmentFileMap, + IndexIO indexIO + ) throws IOException + { + final List segments = new ArrayList<>(); + + for (TimelineObjectHolder timelineSegment : timelineSegments) { + final PartitionHolder partitionHolder = timelineSegment.getObject(); + for (PartitionChunk chunk : partitionHolder) { + final DataSegment segment = chunk.getObject(); + segments.add( + indexIO.loadIndex( + Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier()) + ) + ); + } + } + + return segments; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java deleted file mode 100644 index 8ef60ffb2b51..000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTaskUtils.java +++ /dev/null @@ -1,81 +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.indexing.common.task; - -import com.google.common.base.Preconditions; -import io.druid.java.util.common.guava.Comparators; -import io.druid.segment.IndexIO; -import io.druid.segment.QueryableIndex; -import io.druid.segment.loading.SegmentLoadingException; -import io.druid.timeline.DataSegment; -import io.druid.timeline.TimelineObjectHolder; -import io.druid.timeline.VersionedIntervalTimeline; -import io.druid.timeline.partition.PartitionChunk; -import io.druid.timeline.partition.PartitionHolder; -import org.joda.time.Interval; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class CompactionTaskUtils -{ - public static List> toTimelineSegments( - List usedSegments, - Interval interval - ) throws IOException, SegmentLoadingException - { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - Comparators.naturalNullsFirst() - ); - - for (DataSegment segment : usedSegments) { - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - } - return timeline.lookup(interval); - } - - public static List loadSegments( - List> timelineSegments, - Map segmentFileMap, - IndexIO indexIO - ) throws IOException - { - final List segments = new ArrayList<>(); - - for (TimelineObjectHolder timelineSegment : timelineSegments) { - final PartitionHolder partitionHolder = timelineSegment.getObject(); - for (PartitionChunk chunk : partitionHolder) { - final DataSegment segment = chunk.getObject(); - segments.add( - indexIO.loadIndex( - Preconditions.checkNotNull(segmentFileMap.get(segment), "File for segment %s", segment.getIdentifier()) - ) - ); - } - } - - return segments; - } - - private CompactionTaskUtils() {} -} 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 70570e2d8871..5e97409f500c 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 @@ -127,28 +127,9 @@ public IndexTask( { this( id, + makeGroupId(ingestionSchema), taskResource, ingestionSchema.dataSchema.getDataSource(), - ingestionSchema.ioConfig.appendToExisting, - ingestionSchema, - context - ); - } - - IndexTask( - String id, - TaskResource resource, - String dataSource, - boolean appendToExisting, - IndexIngestionSpec ingestionSchema, - Map context - ) - { - this( - getOrMakeId(id, TYPE, dataSource), - makeGroupId(appendToExisting, dataSource), - resource, - dataSource, ingestionSchema, context ); @@ -174,6 +155,11 @@ public IndexTask( this.ingestionSchema = ingestionSchema; } + private static String makeGroupId(IndexIngestionSpec ingestionSchema) + { + return makeGroupId(ingestionSchema.ioConfig.appendToExisting, ingestionSchema.dataSchema.getDataSource()); + } + private static String makeGroupId(boolean isAppendToExisting, String dataSource) { if (isAppendToExisting) { 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 2277d7097ce3..a3ddaa43c1cd 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 @@ -36,7 +36,6 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.actions.SegmentListUsedAction; -import io.druid.indexing.common.task.CompactionTaskUtils; import io.druid.indexing.common.task.NoopTask; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.filter.DimFilter; @@ -47,6 +46,7 @@ import io.druid.segment.realtime.firehose.WindowedStorageAdapter; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; @@ -145,10 +145,9 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) .getTaskActionClient() .submit(new SegmentListUsedAction(dataSource, interval, null)); final Map segmentFileMap = taskToolbox.fetchSegments(usedSegments); - final List> timeLineSegments = CompactionTaskUtils.toTimelineSegments( - usedSegments, - interval - ); + final List> timeLineSegments = VersionedIntervalTimeline + .forSegments(usedSegments) + .lookup(interval); final List dims; if (dimensions != null) { @@ -216,7 +215,7 @@ public WindowedStorageAdapter apply(final PartitionChunk input) } } - public static List getUniqueDimensions( + private static List getUniqueDimensions( List> timelineSegments, InputRowParser inputRowParser ) @@ -244,7 +243,7 @@ public static List getUniqueDimensions( ); } - public static List getUniqueMetrics(List> timelineSegments) + private static List getUniqueMetrics(List> timelineSegments) { final Set metricsSet = timelineSegments .stream() From 89aeb26f3a10b4483622d303b2e73238288f22da Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Oct 2017 10:57:33 +0900 Subject: [PATCH 05/19] add support for dimensionsSpec --- .../indexing/common/task/CompactionTask.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 9637ec558d59..951d78ec22f7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; @@ -71,6 +72,7 @@ public class CompactionTask extends AbstractTask private static final String TYPE = "compact"; private final Interval interval; + private final DimensionsSpec dimensionsSpec; private final IndexTuningConfig tuningConfig; private final Injector injector; private final IndexIO indexIO; @@ -85,6 +87,7 @@ public CompactionTask( @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("dataSource") final String dataSource, @JsonProperty("interval") final Interval interval, + @JsonProperty("dimensionsSpec") final DimensionsSpec dimensionsSpec, @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, @JsonProperty("context") final Map context, @JacksonInject Injector injector, @@ -94,6 +97,7 @@ public CompactionTask( { super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); this.interval = Preconditions.checkNotNull(interval, "interval"); + this.dimensionsSpec = dimensionsSpec; this.tuningConfig = tuningConfig; this.injector = injector; this.indexIO = indexIO; @@ -140,6 +144,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception toolbox, getDataSource(), interval, + dimensionsSpec, tuningConfig, indexIO, injector, @@ -166,6 +171,7 @@ private static IndexIngestionSpec createIngestionSchema( TaskToolbox toolbox, String dataSource, Interval interval, + DimensionsSpec dimensionsSpec, IndexTuningConfig tuningConfig, IndexIO indexIO, Injector injector, @@ -180,7 +186,7 @@ private static IndexIngestionSpec createIngestionSchema( final Map segmentFileMap = pair.lhs; final List> timelineSegments = pair.rhs; return new IndexIngestionSpec( - createDataSchema(dataSource, interval, indexIO, jsonMapper, timelineSegments, segmentFileMap), + createDataSchema(dataSource, interval, dimensionsSpec, indexIO, jsonMapper, timelineSegments, segmentFileMap), new IndexIOConfig( new IngestSegmentFirehoseFactory( dataSource, @@ -216,6 +222,7 @@ private static Pair, List> timelineSegments, @@ -256,7 +263,13 @@ private static DataSchema createDataSchema( return new DataSchema( dataSource, - ImmutableMap.of("type", "noop"), + dimensionsSpec == null ? ImmutableMap.of("type", "noop") + : ImmutableMap.of( + "type", + "noop", + "parseSpec", + ImmutableMap.of("type", "timeAndDims", "dimensionsSpec", dimensionsSpec) + ), combiningAggregators, granularitySpec, jsonMapper From 91ee6173ebd9dcdb3b20f04dc89899d503e82ec7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Oct 2017 12:12:32 +0900 Subject: [PATCH 06/19] fix getUniqueDims and getUniqueMetics --- .../IngestSegmentFirehoseFactory.java | 93 ++++++++++-------- .../IngestSegmentFirehoseFactoryTest.java | 97 +++++++++++++++++++ 2 files changed, 149 insertions(+), 41 deletions(-) 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 a3ddaa43c1cd..ba570d24cb60 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 @@ -22,12 +22,14 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.google.inject.Injector; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; @@ -48,16 +50,15 @@ import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; -import io.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.IntStream; public class IngestSegmentFirehoseFactory implements FirehoseFactory { @@ -158,12 +159,7 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) dims = getUniqueDimensions(timeLineSegments, inputRowParser); } - final List metricsList; - if (metrics != null) { - metricsList = metrics; - } else { - metricsList = getUniqueMetrics(timeLineSegments); - } + final List metricsList = metrics == null ? getUniqueMetrics(timeLineSegments) : metrics; final List adapters = Lists.newArrayList( Iterables.concat( @@ -215,48 +211,63 @@ public WindowedStorageAdapter apply(final PartitionChunk input) } } - private static List getUniqueDimensions( + @VisibleForTesting + static List getUniqueDimensions( List> timelineSegments, InputRowParser inputRowParser ) { - final Set dimSet = timelineSegments - .stream() - .flatMap(timelineHolder -> { - final PartitionHolder partitionHolder = timelineHolder.getObject(); - final List dims = new ArrayList<>(partitionHolder.size()); - for (PartitionChunk chunk : partitionHolder) { - dims.addAll(chunk.getObject().getDimensions()); + final Set exclusions = inputRowParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(); + final BiMap uniqueDims = HashBiMap.create(); + + // Here, we try to retain the order of dimensions as they were specified since the order of dimensions may be + // optimized for performance. + // Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more + // frequently, and thus the performance should be optimized for recent ones rather than old ones. + + // timelineSegments are sorted in order of interval + int index = 0; + for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) { + for (PartitionChunk chunk : timelineHolder.getObject()) { + for (String dimension : chunk.getObject().getDimensions()) { + if (!uniqueDims.containsKey(dimension) && !exclusions.contains(dimension)) { + uniqueDims.put(dimension, index++); } - return dims.stream(); - }) - .collect(Collectors.toSet()); + } + } + } - return new ArrayList<>( - Sets.difference( - dimSet, - inputRowParser - .getParseSpec() - .getDimensionsSpec() - .getDimensionExclusions() - ) - ); + final BiMap orderedDims = uniqueDims.inverse(); + return IntStream.range(0, orderedDims.size()) + .mapToObj(orderedDims::get) + .collect(Collectors.toList()); } - private static List getUniqueMetrics(List> timelineSegments) + @VisibleForTesting + static List getUniqueMetrics(List> timelineSegments) { - final Set metricsSet = timelineSegments - .stream() - .flatMap(timelineHolder -> { - final PartitionHolder partitionHolder = timelineHolder.getObject(); - final List metrics = new ArrayList<>(partitionHolder.size()); - for (PartitionChunk chunk : partitionHolder) { - metrics.addAll(chunk.getObject().getMetrics()); + final BiMap uniqueMetrics = HashBiMap.create(); + + // Here, we try to retain the order of metrics as they were specified since the order of metrics may be + // optimized for performance. + // Metrics are extracted from the recent segments to olders because recent segments are likely to be queried more + // frequently, and thus the performance should be optimized for recent ones rather than old ones. + + // timelineSegments are sorted in order of interval + int index = 0; + for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) { + for (PartitionChunk chunk : timelineHolder.getObject()) { + for (String metric : chunk.getObject().getMetrics()) { + if (!uniqueMetrics.containsKey(metric)) { + uniqueMetrics.put(metric, index++); } - return metrics.stream(); - }) - .collect(Collectors.toSet()); + } + } + } - return new ArrayList<>(metricsSet); + final BiMap orderedMetrics = uniqueMetrics.inverse(); + return IntStream.range(0, orderedMetrics.size()) + .mapToObj(orderedMetrics::get) + .collect(Collectors.toList()); } } 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 bbdf264741e9..befe6b63a4b3 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 @@ -38,6 +38,7 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.NoopInputRowParser; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.data.input.impl.TimestampSpec; import io.druid.guice.GuiceAnnotationIntrospector; @@ -81,7 +82,11 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.partition.NumberedPartitionChunk; import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.PartitionHolder; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.AfterClass; @@ -102,6 +107,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * @@ -533,6 +540,96 @@ public void simpleFirehoseReadingTest() throws IOException Assert.assertEquals((int) MAX_SHARD_NUMBER * MAX_ROWS, (int) rowcount); } + @Test + public void testGetUniqueDimensionsAndMetrics() + { + final int numSegmentsPerPartitionChunk = 5; + final int numPartitionChunksPerTimelineObject = 10; + final int numSegments = numSegmentsPerPartitionChunk * numPartitionChunksPerTimelineObject; + final List segments = new ArrayList<>(numSegments); + final Interval interval = Intervals.of("2017-01-01/2017-01-02"); + final String version = "1"; + + final List> timelineSegments = new ArrayList<>(); + for (int i = 0; i < numPartitionChunksPerTimelineObject; i++) { + final List> chunks = new ArrayList<>(); + for (int j = 0; j < numSegmentsPerPartitionChunk; j++) { + final List dims = IntStream.range(i, i + numSegmentsPerPartitionChunk) + .mapToObj(suffix -> "dim" + suffix) + .collect(Collectors.toList()); + final List metrics = IntStream.range(i, i + numSegmentsPerPartitionChunk) + .mapToObj(suffix -> "met" + suffix) + .collect(Collectors.toList()); + final DataSegment segment = new DataSegment( + "ds", + interval, + version, + ImmutableMap.of(), + dims, + metrics, + new NumberedShardSpec(numPartitionChunksPerTimelineObject, i), + 1, + 1 + ); + segments.add(segment); + + final PartitionChunk partitionChunk = new NumberedPartitionChunk<>( + i, + numPartitionChunksPerTimelineObject, + segment + ); + chunks.add(partitionChunk); + } + final TimelineObjectHolder timelineHolder = new TimelineObjectHolder<>( + interval, + version, + new PartitionHolder<>(chunks) + ); + timelineSegments.add(timelineHolder); + } + + final String[] expectedDims = new String[]{ + "dim9", + "dim10", + "dim11", + "dim12", + "dim13", + "dim8", + "dim7", + "dim6", + "dim5", + "dim4", + "dim3", + "dim2", + "dim1", + "dim0" + }; + final String[] expectedMetrics = new String[]{ + "met9", + "met10", + "met11", + "met12", + "met13", + "met8", + "met7", + "met6", + "met5", + "met4", + "met3", + "met2", + "met1", + "met0" + }; + Assert.assertEquals( + Arrays.asList(expectedDims), + IngestSegmentFirehoseFactory.getUniqueDimensions(timelineSegments, new NoopInputRowParser(null)) + ); + Assert.assertEquals( + Arrays.asList(expectedMetrics), + IngestSegmentFirehoseFactory.getUniqueMetrics(timelineSegments) + ); + } + private static ServiceEmitter newMockEmitter() { return new NoopServiceEmitter(); From 075729454df46a261b98424e5c14d27af91fd730 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Oct 2017 11:00:15 +0900 Subject: [PATCH 07/19] find unique dimensionsSpec --- docs/content/ingestion/tasks.md | 1 + .../indexing/common/task/CompactionTask.java | 167 +++++++++++++++--- .../IngestSegmentFirehoseFactory.java | 16 +- .../io/druid/segment/DimensionHandler.java | 9 + .../druid/segment/StringDimensionHandler.java | 6 + 5 files changed, 162 insertions(+), 37 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index fe2237cd397e..847d1c0a4121 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -274,6 +274,7 @@ An example of compaction task is ``` This compaction task merges _all segments_ of the interval `2017-01-01/2018-01-01` into a _single segment_. +To merge each day's worth of data into a separate segment, you can submit multiple `compact` tasks, one for each day. They will run in parallel. A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters. For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html) and `dimensionsSpec` and `metricsSpec` diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 951d78ec22f7..50ecaf9f09c5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -25,10 +25,21 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.inject.Injector; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.DoubleDimensionSchema; +import io.druid.data.input.impl.FloatDimensionSchema; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.LongDimensionSchema; +import io.druid.data.input.impl.NoopInputRowParser; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.TimeAndDimsParseSpec; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; @@ -41,10 +52,14 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.NoneGranularity; import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.logger.Logger; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.DimensionHandler; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec; @@ -60,11 +75,13 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; import java.util.stream.Collectors; +import java.util.stream.IntStream; public class CompactionTask extends AbstractTask { @@ -72,7 +89,6 @@ public class CompactionTask extends AbstractTask private static final String TYPE = "compact"; private final Interval interval; - private final DimensionsSpec dimensionsSpec; private final IndexTuningConfig tuningConfig; private final Injector injector; private final IndexIO indexIO; @@ -87,7 +103,6 @@ public CompactionTask( @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("dataSource") final String dataSource, @JsonProperty("interval") final Interval interval, - @JsonProperty("dimensionsSpec") final DimensionsSpec dimensionsSpec, @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, @JsonProperty("context") final Map context, @JacksonInject Injector injector, @@ -97,7 +112,6 @@ public CompactionTask( { super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); this.interval = Preconditions.checkNotNull(interval, "interval"); - this.dimensionsSpec = dimensionsSpec; this.tuningConfig = tuningConfig; this.injector = injector; this.indexIO = indexIO; @@ -144,7 +158,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception toolbox, getDataSource(), interval, - dimensionsSpec, tuningConfig, indexIO, injector, @@ -171,7 +184,6 @@ private static IndexIngestionSpec createIngestionSchema( TaskToolbox toolbox, String dataSource, Interval interval, - DimensionsSpec dimensionsSpec, IndexTuningConfig tuningConfig, IndexIO indexIO, Injector injector, @@ -185,15 +197,24 @@ private static IndexIngestionSpec createIngestionSchema( ); final Map segmentFileMap = pair.lhs; final List> timelineSegments = pair.rhs; + final DataSchema dataSchema = createDataSchema( + dataSource, + interval, + indexIO, + jsonMapper, + timelineSegments, + segmentFileMap + ); return new IndexIngestionSpec( - createDataSchema(dataSource, interval, dimensionsSpec, indexIO, jsonMapper, timelineSegments, segmentFileMap), + dataSchema, new IndexIOConfig( new IngestSegmentFirehoseFactory( dataSource, interval, - null, // no filter - null, // null means all unique dimensions - null, // null means all unique metrics + null,// no filter + // set dimensions and metrics names to make sure that the generated dataSchema is used for the firehose + dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(), + Arrays.stream(dataSchema.getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toList()), injector, indexIO ), @@ -209,9 +230,8 @@ private static Pair, List usedSegments = toolbox - .getTaskActionClient() - .submit(new SegmentListUsedAction(dataSource, interval, null)); + final List usedSegments = toolbox.getTaskActionClient() + .submit(new SegmentListUsedAction(dataSource, interval, null)); final Map segmentFileMap = toolbox.fetchSegments(usedSegments); final List> timelineSegments = VersionedIntervalTimeline .forSegments(usedSegments) @@ -222,7 +242,6 @@ private static Pair, List> timelineSegments, @@ -231,22 +250,20 @@ private static DataSchema createDataSchema( throws IOException, SegmentLoadingException { // find metadata for interval - final List segments = loadSegments(timelineSegments, segmentFileMap, indexIO); - - // set rollup only if rollup is set for all segments - final boolean rollup = segments.stream().allMatch(index -> index.getMetadata().isRollup()); + final List queryableIndices = loadSegments(timelineSegments, segmentFileMap, indexIO); // find merged aggregators - final List aggregatorFactories = segments.stream() - .map(index -> index.getMetadata().getAggregators()) - .collect(Collectors.toList()); + final List aggregatorFactories = queryableIndices + .stream() + .map(index -> index.getMetadata().getAggregators()) + .collect(Collectors.toList()); final AggregatorFactory[] mergedAggregators = AggregatorFactory.mergeAggregators(aggregatorFactories); if (mergedAggregators == null) { throw new ISE("Failed to merge aggregators[%s]", aggregatorFactories); } - // conver to combining aggregators + // convert to combining aggregators final List combiningAggregatorList = Arrays.stream(mergedAggregators) .map(AggregatorFactory::getCombiningFactory) .collect(Collectors.toList()); @@ -255,27 +272,84 @@ private static DataSchema createDataSchema( ); // find granularity spec + // set rollup only if rollup is set for all segments + final boolean rollup = queryableIndices.stream().allMatch(index -> index.getMetadata().isRollup()); final GranularitySpec granularitySpec = new ArbitraryGranularitySpec( new NoneGranularity(), rollup, ImmutableList.of(interval) ); + // find unique dimensions + final DimensionsSpec dimensionsSpec = createDimensionsSpec(queryableIndices); + final InputRowParser parser = new NoopInputRowParser(new TimeAndDimsParseSpec(null, dimensionsSpec)); + return new DataSchema( dataSource, - dimensionsSpec == null ? ImmutableMap.of("type", "noop") - : ImmutableMap.of( - "type", - "noop", - "parseSpec", - ImmutableMap.of("type", "timeAndDims", "dimensionsSpec", dimensionsSpec) - ), + jsonMapper.convertValue(parser, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT), combiningAggregators, granularitySpec, jsonMapper ); } + private static DimensionsSpec createDimensionsSpec(List queryableIndices) + { + final BiMap uniqueDims = HashBiMap.create(); + final Map dimensionSchemaMap = new HashMap<>(); + + // Here, we try to retain the order of dimensions as they were specified since the order of dimensions may be + // optimized for performance. + // Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more + // frequently, and thus the performance should be optimized for recent ones rather than old ones. + + // timelineSegments are sorted in order of interval + int index = 0; + for (QueryableIndex queryableIndex : Lists.reverse(queryableIndices)) { + final Map dimensionHandlerMap = queryableIndex.getDimensionHandlers(); + + for (String dimension : queryableIndex.getAvailableDimensions()) { + final Column column = Preconditions.checkNotNull( + queryableIndex.getColumn(dimension), + "Cannot find column for dimension[%s]", + dimension + ); + + if (!uniqueDims.containsKey(dimension)) { + final DimensionHandler dimensionHandler = Preconditions.checkNotNull( + dimensionHandlerMap.get(dimension), + "Cannot find dimensionHandler for dimension[%s]", + dimension + ); + + uniqueDims.put(dimension, index++); + dimensionSchemaMap.put( + dimension, + createDimensionSchema( + column.getCapabilities().getType(), + dimension, + dimensionHandler.getMultivalueHandling() + ) + ); + } + } + } + + final BiMap orderedDims = uniqueDims.inverse(); + final List dimensionSchemas = IntStream.range(0, orderedDims.size()) + .mapToObj(i -> { + final String dimName = orderedDims.get(i); + return Preconditions.checkNotNull( + dimensionSchemaMap.get(dimName), + "Cannot find dimension[%s] from dimensionSchemaMap", + dimName + ); + }) + .collect(Collectors.toList()); + + return new DimensionsSpec(dimensionSchemas, null, null); + } + private static List loadSegments( List> timelineSegments, Map segmentFileMap, @@ -298,4 +372,39 @@ private static List loadSegments( return segments; } + + private static DimensionSchema createDimensionSchema( + ValueType type, + String name, + MultiValueHandling multiValueHandling + ) + { + switch (type) { + case FLOAT: + Preconditions.checkArgument( + multiValueHandling == null, + "multi-value dimension [%s] is not supported for float type yet", + name + ); + return new FloatDimensionSchema(name); + case LONG: + Preconditions.checkArgument( + multiValueHandling == null, + "multi-value dimension [%s] is not supported for long type yet", + name + ); + return new LongDimensionSchema(name); + case DOUBLE: + Preconditions.checkArgument( + multiValueHandling == null, + "multi-value dimension [%s] is not supported for double type yet", + name + ); + return new DoubleDimensionSchema(name); + case STRING: + return new StringDimensionSchema(name, multiValueHandling); + default: + throw new ISE("Unsupported value type[%s] for dimension[%s]", type, name); + } + } } 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 ba570d24cb60..41e5bf5476cd 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 @@ -156,7 +156,10 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) } else if (inputRowParser.getParseSpec().getDimensionsSpec().hasCustomDimensions()) { dims = inputRowParser.getParseSpec().getDimensionsSpec().getDimensionNames(); } else { - dims = getUniqueDimensions(timeLineSegments, inputRowParser); + dims = getUniqueDimensions( + timeLineSegments, + inputRowParser.getParseSpec().getDimensionsSpec().getDimensionExclusions() + ); } final List metricsList = metrics == null ? getUniqueMetrics(timeLineSegments) : metrics; @@ -214,10 +217,9 @@ public WindowedStorageAdapter apply(final PartitionChunk input) @VisibleForTesting static List getUniqueDimensions( List> timelineSegments, - InputRowParser inputRowParser + Set excludeDimensions ) { - final Set exclusions = inputRowParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(); final BiMap uniqueDims = HashBiMap.create(); // Here, we try to retain the order of dimensions as they were specified since the order of dimensions may be @@ -230,7 +232,7 @@ static List getUniqueDimensions( for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) { for (PartitionChunk chunk : timelineHolder.getObject()) { for (String dimension : chunk.getObject().getDimensions()) { - if (!uniqueDims.containsKey(dimension) && !exclusions.contains(dimension)) { + if (!uniqueDims.containsKey(dimension) && !excludeDimensions.contains(dimension)) { uniqueDims.put(dimension, index++); } } @@ -248,10 +250,8 @@ static List getUniqueMetrics(List uniqueMetrics = HashBiMap.create(); - // Here, we try to retain the order of metrics as they were specified since the order of metrics may be - // optimized for performance. - // Metrics are extracted from the recent segments to olders because recent segments are likely to be queried more - // frequently, and thus the performance should be optimized for recent ones rather than old ones. + // Here, we try to retain the order of metrics as they were specified. Metrics are extracted from the recent + // segments to olders. // timelineSegments are sorted in order of interval int index = 0; diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index f0c24cd80794..a44e1d78efa9 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -19,6 +19,7 @@ package io.druid.segment; +import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.IOPeon; @@ -69,6 +70,14 @@ public interface DimensionHandler */ String getDimensionName(); + /** + * Get {@link MultiValueHandling} for the column associated with this handler. + * Only string columns can have {@link MultiValueHandling} currently. + */ + default MultiValueHandling getMultivalueHandling() + { + return null; + } /** * Creates a new DimensionIndexer, a per-dimension object responsible for processing ingested rows in-memory, used diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index bfe510c6f781..bfb993afbb40 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -50,6 +50,12 @@ public String getDimensionName() return dimensionName; } + @Override + public MultiValueHandling getMultivalueHandling() + { + return multiValueHandling; + } + @Override public int getLengthOfEncodedKeyComponent(int[] dimVals) { From ffe21e7af4b2725caea5236724080dab36f1ba5a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Oct 2017 11:09:44 +0900 Subject: [PATCH 08/19] fix compilation --- .../java/io/druid/indexing/common/task/CompactionTask.java | 2 +- .../indexing/firehose/IngestSegmentFirehoseFactory.java | 6 ++++-- .../indexing/firehose/IngestSegmentFirehoseFactoryTest.java | 3 +-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 50ecaf9f09c5..87af76408341 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -211,7 +211,7 @@ private static IndexIngestionSpec createIngestionSchema( new IngestSegmentFirehoseFactory( dataSource, interval, - null,// no filter + null, // no filter // set dimensions and metrics names to make sure that the generated dataSchema is used for the firehose dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(), Arrays.stream(dataSchema.getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toList()), 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 41e5bf5476cd..0dc40eef1444 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 @@ -52,6 +52,7 @@ import io.druid.timeline.partition.PartitionChunk; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; @@ -217,7 +218,7 @@ public WindowedStorageAdapter apply(final PartitionChunk input) @VisibleForTesting static List getUniqueDimensions( List> timelineSegments, - Set excludeDimensions + @Nullable Set excludeDimensions ) { final BiMap uniqueDims = HashBiMap.create(); @@ -232,7 +233,8 @@ static List getUniqueDimensions( for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) { for (PartitionChunk chunk : timelineHolder.getObject()) { for (String dimension : chunk.getObject().getDimensions()) { - if (!uniqueDims.containsKey(dimension) && !excludeDimensions.contains(dimension)) { + if (!uniqueDims.containsKey(dimension) && + (excludeDimensions == null || !excludeDimensions.contains(dimension))) { uniqueDims.put(dimension, index++); } } 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 befe6b63a4b3..ad4871b3cce1 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 @@ -38,7 +38,6 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; -import io.druid.data.input.impl.NoopInputRowParser; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.data.input.impl.TimestampSpec; import io.druid.guice.GuiceAnnotationIntrospector; @@ -622,7 +621,7 @@ public void testGetUniqueDimensionsAndMetrics() }; Assert.assertEquals( Arrays.asList(expectedDims), - IngestSegmentFirehoseFactory.getUniqueDimensions(timelineSegments, new NoopInputRowParser(null)) + IngestSegmentFirehoseFactory.getUniqueDimensions(timelineSegments, null) ); Assert.assertEquals( Arrays.asList(expectedMetrics), From 1fcdf4aee5094b8c361fd96742795cd0d025507a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Oct 2017 16:31:11 +0900 Subject: [PATCH 09/19] add unit test --- .../data/input/impl/DimensionSchema.java | 23 +- .../data/input/impl/NoopInputRowParser.java | 1 + .../variance/VarianceAggregatorFactory.java | 2 +- .../indexing/common/task/CompactionTask.java | 23 +- .../druid/indexing/common/task/IndexTask.java | 79 ++++ .../common/task/CompactionTaskTest.java | 432 ++++++++++++++++++ .../query/aggregation/AggregatorFactory.java | 19 +- .../DoubleMaxAggregatorFactory.java | 26 -- .../DoubleMinAggregatorFactory.java | 26 -- .../DoubleSumAggregatorFactory.java | 27 -- .../SimpleDoubleAggregatorFactory.java | 25 + .../CardinalityAggregatorFactory.java | 7 - .../first/DoubleFirstAggregatorFactory.java | 11 - .../first/FloatFirstAggregatorFactory.java | 11 - .../first/LongFirstAggregatorFactory.java | 11 - .../last/DoubleLastAggregatorFactory.java | 11 - .../last/FloatLastAggregatorFactory.java | 11 - .../last/LongLastAggregatorFactory.java | 11 - .../segment/incremental/IncrementalIndex.java | 2 +- 19 files changed, 583 insertions(+), 175 deletions(-) create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java diff --git a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java index 56f9182ec958..045dcb776bd1 100644 --- a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java @@ -29,6 +29,8 @@ import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.StringUtils; +import java.util.Objects; + /** */ @PublicApi @@ -149,13 +151,30 @@ public boolean equals(Object o) DimensionSchema that = (DimensionSchema) o; - return name.equals(that.name); + if (!name.equals(that.name)) { + return false; + } + + if (!getValueType().equals(that.getValueType())) { + return false; + } + return Objects.equals(multiValueHandling, that.multiValueHandling); } @Override public int hashCode() { - return name.hashCode(); + return Objects.hash(name, getValueType(), multiValueHandling); + } + + @Override + public String toString() + { + return "DimensionSchema{" + + "name='" + name + "'" + + ", valueType='" + getValueType() + "'" + + ", multiValueHandling='" + getMultiValueHandling() + "'" + + "}"; } } diff --git a/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java index 772024223c85..7b41391309a9 100644 --- a/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java +++ b/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java @@ -43,6 +43,7 @@ public InputRow parse(InputRow input) return input; } + @JsonProperty @Override public ParseSpec getParseSpec() { 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 3874a981b0eb..d8ec09e7a158 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 @@ -190,7 +190,7 @@ public List getRequiredColumns() @Override public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException { - if (Objects.equals(getName(), other.getName()) && this.getClass() == other.getClass()) { + if (Objects.equals(getName(), other.getName()) && other instanceof VarianceAggregatorFactory) { return getCombiningFactory(); } else { throw new AggregatorFactoryNotMergeableException(this, other); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 87af76408341..c299dc9353dd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; @@ -91,7 +92,6 @@ public class CompactionTask extends AbstractTask private final Interval interval; private final IndexTuningConfig tuningConfig; private final Injector injector; - private final IndexIO indexIO; private final ObjectMapper jsonMapper; @JsonIgnore @@ -106,7 +106,6 @@ public CompactionTask( @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, @JsonProperty("context") final Map context, @JacksonInject Injector injector, - @JacksonInject IndexIO indexIO, @JacksonInject ObjectMapper jsonMapper ) { @@ -114,7 +113,6 @@ public CompactionTask( this.interval = Preconditions.checkNotNull(interval, "interval"); this.tuningConfig = tuningConfig; this.injector = injector; - this.indexIO = indexIO; this.jsonMapper = jsonMapper; } @@ -159,7 +157,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception getDataSource(), interval, tuningConfig, - indexIO, injector, jsonMapper ); @@ -180,12 +177,12 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception return indexTaskSpec.run(toolbox); } - private static IndexIngestionSpec createIngestionSchema( + @VisibleForTesting + static IndexIngestionSpec createIngestionSchema( TaskToolbox toolbox, String dataSource, Interval interval, IndexTuningConfig tuningConfig, - IndexIO indexIO, Injector injector, ObjectMapper jsonMapper ) throws IOException, SegmentLoadingException @@ -200,7 +197,7 @@ private static IndexIngestionSpec createIngestionSchema( final DataSchema dataSchema = createDataSchema( dataSource, interval, - indexIO, + toolbox.getIndexIO(), jsonMapper, timelineSegments, segmentFileMap @@ -216,7 +213,7 @@ private static IndexIngestionSpec createIngestionSchema( dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(), Arrays.stream(dataSchema.getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toList()), injector, - indexIO + toolbox.getIndexIO() ), false ), @@ -263,14 +260,6 @@ private static DataSchema createDataSchema( throw new ISE("Failed to merge aggregators[%s]", aggregatorFactories); } - // convert to combining aggregators - final List combiningAggregatorList = Arrays.stream(mergedAggregators) - .map(AggregatorFactory::getCombiningFactory) - .collect(Collectors.toList()); - final AggregatorFactory[] combiningAggregators = combiningAggregatorList.toArray( - new AggregatorFactory[combiningAggregatorList.size()] - ); - // find granularity spec // set rollup only if rollup is set for all segments final boolean rollup = queryableIndices.stream().allMatch(index -> index.getMetadata().isRollup()); @@ -287,7 +276,7 @@ private static DataSchema createDataSchema( return new DataSchema( dataSource, jsonMapper.convertValue(parser, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT), - combiningAggregators, + mergedAggregators, granularitySpec, jsonMapper ); 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 5e97409f500c..d740f92aba5c 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 @@ -96,6 +96,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.SortedSet; import java.util.TreeMap; @@ -1150,5 +1151,83 @@ public Period getIntermediatePersistPeriod() { return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final IndexTuningConfig that = (IndexTuningConfig) o; + + if (!Objects.equals(targetPartitionSize, that.targetPartitionSize)) { + return false; + } + + if (maxRowsInMemory != that.maxRowsInMemory) { + return false; + } + + if (maxTotalRows != that.maxTotalRows) { + return false; + } + + if (!Objects.equals(numShards, that.numShards)) { + return false; + } + + if (!Objects.equals(indexSpec, that.indexSpec)) { + return false; + } + + if (!Objects.equals(basePersistDirectory, that.basePersistDirectory)) { + return false; + } + + if (maxPendingPersists != that.maxPendingPersists) { + return false; + } + + if (forceExtendableShardSpecs != that.forceExtendableShardSpecs) { + return false; + } + + if (forceGuaranteedRollup != that.forceGuaranteedRollup) { + return false; + } + + if (reportParseExceptions != that.reportParseExceptions) { + return false; + } + + if (publishTimeout != that.publishTimeout) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return Objects.hash( + targetPartitionSize, + maxRowsInMemory, + maxTotalRows, + numShards, + indexSpec, + basePersistDirectory, + maxPendingPersists, + forceExtendableShardSpecs, + forceGuaranteedRollup, + reportParseExceptions, + publishTimeout + ); + } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java new file mode 100644 index 000000000000..3c9cb043bbeb --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -0,0 +1,432 @@ +/* + * 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.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; +import io.druid.data.input.impl.DoubleDimensionSchema; +import io.druid.data.input.impl.FloatDimensionSchema; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.LongDimensionSchema; +import io.druid.data.input.impl.NoopInputRowParser; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.guice.GuiceInjectors; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.actions.TaskAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.IndexTask.IndexIOConfig; +import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongMaxAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.first.FloatFirstAggregatorFactory; +import io.druid.query.aggregation.last.DoubleLastAggregatorFactory; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMergerV9; +import io.druid.segment.IndexSpec; +import io.druid.segment.Metadata; +import io.druid.segment.QueryableIndex; +import io.druid.segment.SimpleQueryableIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy; +import io.druid.segment.data.CompressionFactory.LongEncodingStrategy; +import io.druid.segment.data.ListIndexed; +import io.druid.segment.data.RoaringBitmapSerdeFactory; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.stream.Collectors; + +public class CompactionTaskTest +{ + private static final String DATA_SOURCE = "dataSource"; + private static final String TIMESTAMP_COLUMN = "timestamp"; + private static final Interval INTERVAL = Intervals.of("2017-01-01/2018-01-01"); + private static final IndexTuningConfig TUNING_CONFIG = createTuningConfig(); + + private static Map DIMENSIONS; + private static Map AGGREGATORS; + private static Map SEGMENT_MAP; + private static ObjectMapper objectMapper = new DefaultObjectMapper(); + private static TaskToolbox toolbox; + + @BeforeClass + public static void setupClass() + { + DIMENSIONS = new HashMap<>(); + AGGREGATORS = new HashMap<>(); + + DIMENSIONS.put(Column.TIME_COLUMN_NAME, new LongDimensionSchema(Column.TIME_COLUMN_NAME)); + DIMENSIONS.put(TIMESTAMP_COLUMN, new LongDimensionSchema(TIMESTAMP_COLUMN)); + for (int i = 0; i < 5; i++) { + final StringDimensionSchema schema = new StringDimensionSchema( + "string_dim_" + i, + MultiValueHandling.SORTED_ARRAY + ); + DIMENSIONS.put(schema.getName(), schema); + } + for (int i = 0; i < 5; i++) { + final LongDimensionSchema schema = new LongDimensionSchema("long_dim_" + i); + DIMENSIONS.put(schema.getName(), schema); + } + for (int i = 0; i < 5; i++) { + final FloatDimensionSchema schema = new FloatDimensionSchema("float_dim_" + i); + DIMENSIONS.put(schema.getName(), schema); + } + for (int i = 0; i < 5; i++) { + final DoubleDimensionSchema schema = new DoubleDimensionSchema("double_dim_" + i); + DIMENSIONS.put(schema.getName(), schema); + } + + AGGREGATORS.put("agg_0", new CountAggregatorFactory("agg_0")); + AGGREGATORS.put("agg_1", new LongSumAggregatorFactory("agg_1", "long_dim_1")); + AGGREGATORS.put("agg_2", new LongMaxAggregatorFactory("agg_2", "long_dim_2")); + AGGREGATORS.put("agg_3", new FloatFirstAggregatorFactory("agg_3", "float_dim_3")); + AGGREGATORS.put("agg_4", new DoubleLastAggregatorFactory("agg_4", "double_dim_4")); + + SEGMENT_MAP = new HashMap<>(5); + for (int i = 0; i < 5; i++) { + SEGMENT_MAP.put( + new DataSegment( + DATA_SOURCE, + INTERVAL, + "version", + ImmutableMap.of(), + findDimensions(i), + new ArrayList<>(AGGREGATORS.keySet()), + new NumberedShardSpec(i, 5), + 0, + 1 + ), + new File("file_" + i) + ); + } + + toolbox = new TestTaskToolbox( + new TestTaskActionClient(new ArrayList<>(SEGMENT_MAP.keySet())), + new TestIndexIO(objectMapper, SEGMENT_MAP), + SEGMENT_MAP + ); + } + + private static List findDimensions(int index) + { + final List dimensions = new ArrayList<>(); + dimensions.add(TIMESTAMP_COLUMN); + for (int i = index; i < Math.min(index + 3, 5); i++) { + dimensions.add("string_dim_" + i); + dimensions.add("long_dim_" + i); + dimensions.add("float_dim_" + i); + dimensions.add("double_dim_" + i); + } + return dimensions; + } + + private static IndexTuningConfig createTuningConfig() + { + return new IndexTuningConfig( + 5000000, + 500000, + 1000000, + null, + null, + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.LZ4, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + 5000, + true, + false, + true, + false, + 100L + ); + } + + @Before + public void setup() throws IOException + { + + } + + @Test + public void testCreateIngestionSchema() throws IOException, SegmentLoadingException + { + final IndexIngestionSpec ingestionSchema = CompactionTask.createIngestionSchema( + toolbox, + DATA_SOURCE, + INTERVAL, + TUNING_CONFIG, + GuiceInjectors.makeStartupInjector(), + objectMapper + ); + + // assert dataSchema + final DataSchema dataSchema = ingestionSchema.getDataSchema(); + Assert.assertEquals(DATA_SOURCE, dataSchema.getDataSource()); + + final InputRowParser parser = objectMapper.convertValue(dataSchema.getParser(), InputRowParser.class); + Assert.assertTrue(parser instanceof NoopInputRowParser); + Assert.assertTrue(parser.getParseSpec() instanceof TimeAndDimsParseSpec); + Assert.assertEquals( + new HashSet<>(Sets.difference( + new HashSet<>(DIMENSIONS.values()), ImmutableSet.of(new LongDimensionSchema(Column.TIME_COLUMN_NAME))) + ), + new HashSet<>(parser.getParseSpec().getDimensionsSpec().getDimensions()) + ); + final Set expectedAggregators = AGGREGATORS.values() + .stream() + .map(AggregatorFactory::getCombiningFactory) + .collect(Collectors.toSet()); + Assert.assertEquals(expectedAggregators, new HashSet<>(Arrays.asList(dataSchema.getAggregators()))); + Assert.assertEquals( + new ArbitraryGranularitySpec(Granularities.NONE, false, ImmutableList.of(INTERVAL)), + dataSchema.getGranularitySpec() + ); + + // assert ioConfig + final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); + Assert.assertFalse(ioConfig.isAppendToExisting()); + final FirehoseFactory firehoseFactory = ioConfig.getFirehoseFactory(); + Assert.assertTrue(firehoseFactory instanceof IngestSegmentFirehoseFactory); + final IngestSegmentFirehoseFactory ingestSegmentFirehoseFactory = (IngestSegmentFirehoseFactory) firehoseFactory; + Assert.assertEquals(DATA_SOURCE, ingestSegmentFirehoseFactory.getDataSource()); + Assert.assertEquals(INTERVAL, ingestSegmentFirehoseFactory.getInterval()); + Assert.assertNull(ingestSegmentFirehoseFactory.getDimensionsFilter()); + // check the order of dimensions + Assert.assertEquals( + Lists.newArrayList( + "timestamp", + "string_dim_4", + "long_dim_4", + "float_dim_4", + "double_dim_4", + "string_dim_3", + "long_dim_3", + "float_dim_3", + "double_dim_3", + "string_dim_2", + "long_dim_2", + "float_dim_2", + "double_dim_2", + "string_dim_1", + "long_dim_1", + "float_dim_1", + "double_dim_1", + "string_dim_0", + "long_dim_0", + "float_dim_0", + "double_dim_0" + ), + ingestSegmentFirehoseFactory.getDimensions() + ); + // check the order of metrics + Assert.assertEquals( + Lists.newArrayList("agg_4", "agg_3", "agg_2", "agg_1", "agg_0"), + ingestSegmentFirehoseFactory.getMetrics() + ); + + // assert tuningConfig + Assert.assertEquals(createTuningConfig(), ingestionSchema.getTuningConfig()); + } + + private static class TestTaskToolbox extends TaskToolbox + { + private final Map segmentFileMap; + + TestTaskToolbox( + TaskActionClient taskActionClient, + IndexIO indexIO, + Map segmentFileMap + ) + { + super( + null, + taskActionClient, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + indexIO, + null, + null, + new IndexMergerV9(objectMapper, indexIO), + null, + null, + null, + null + ); + this.segmentFileMap = segmentFileMap; + } + + @Override + public Map fetchSegments(List segments) + throws SegmentLoadingException + { + final Map submap = new HashMap<>(segments.size()); + for (DataSegment segment : segments) { + final File file = Preconditions.checkNotNull(segmentFileMap.get(segment)); + submap.put(segment, file); + } + return submap; + } + } + + private static class TestTaskActionClient implements TaskActionClient + { + private final List segments; + + TestTaskActionClient(List segments) + { + this.segments = segments; + } + + @Override + public RetType submit(TaskAction taskAction) throws IOException + { + if (!(taskAction instanceof SegmentListUsedAction)) { + throw new ISE("action[%s] is not supported", taskAction); + } + return (RetType) segments; + } + } + + private static class TestIndexIO extends IndexIO + { + private final Map queryableIndexMap; + + TestIndexIO( + ObjectMapper mapper, + Map segmentFileMap + ) + { + super(mapper, () -> 0); + + queryableIndexMap = new HashMap<>(segmentFileMap.size()); + for (Entry entry : segmentFileMap.entrySet()) { + final DataSegment segment = entry.getKey(); + final List columnNames = new ArrayList<>(segment.getDimensions().size() + segment.getMetrics().size()); + columnNames.add(Column.TIME_COLUMN_NAME); + columnNames.addAll(segment.getDimensions()); + columnNames.addAll(segment.getMetrics()); + final Map columnMap = new HashMap<>(columnNames.size()); + final List aggregatorFactories = new ArrayList<>(segment.getMetrics().size()); + + for (String columnName : columnNames) { + if (DIMENSIONS.containsKey(columnName)) { + columnMap.put(columnName, createColumn(DIMENSIONS.get(columnName))); + } else if (AGGREGATORS.containsKey(columnName)) { + columnMap.put(columnName, createColumn(AGGREGATORS.get(columnName))); + aggregatorFactories.add(AGGREGATORS.get(columnName)); + } + } + + final Metadata metadata = new Metadata(); + metadata.setAggregators(aggregatorFactories.toArray(new AggregatorFactory[aggregatorFactories.size()])); + metadata.setRollup(false); + + queryableIndexMap.put( + entry.getValue(), + new SimpleQueryableIndex( + segment.getInterval(), + new ListIndexed<>(columnNames, String.class), + new ListIndexed<>(segment.getDimensions(), String.class), + null, + columnMap, + null, + metadata + ) + ); + } + } + + @Override + public QueryableIndex loadIndex(File file) throws IOException + { + return queryableIndexMap.get(file); + } + } + + private static Column createColumn(DimensionSchema dimensionSchema) + { + return new ColumnBuilder() + .setType(IncrementalIndex.TYPE_MAP.get(dimensionSchema.getValueType())) + .setDictionaryEncodedColumn(() -> null) + .setBitmapIndex(() -> null) + .build(); + } + + private static Column createColumn(AggregatorFactory aggregatorFactory) + { + return new ColumnBuilder() + .setType(ValueType.fromString(aggregatorFactory.getTypeName())) + .build(); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index f4e67c34e26b..8fab79262b50 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -26,6 +26,7 @@ import io.druid.segment.ColumnSelectorFactory; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Comparator; import java.util.LinkedHashMap; import java.util.List; @@ -92,7 +93,12 @@ public AggregateCombiner makeAggregateCombiner() */ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException { - throw new UOE("[%s] does not implement getMergingFactory(..)", this.getClass().getName()); + final AggregatorFactory combiningFactory = this.getCombiningFactory(); + if (other.getName().equals(this.getName()) && combiningFactory.equals(other.getCombiningFactory())) { + return combiningFactory; + } else { + throw new AggregatorFactoryNotMergeableException(this, other); + } } /** @@ -153,6 +159,13 @@ public static AggregatorFactory[] mergeAggregators(List agg return null; } + if (aggregatorsList.size() == 1) { + final AggregatorFactory[] aggregatorFactories = aggregatorsList.get(0); + final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; + Arrays.setAll(combiningFactories, i -> aggregatorFactories[i].getCombiningFactory()); + return combiningFactories; + } + Map mergedAggregators = new LinkedHashMap<>(); for (AggregatorFactory[] aggregators : aggregatorsList) { @@ -163,7 +176,9 @@ public static AggregatorFactory[] mergeAggregators(List agg if (mergedAggregators.containsKey(name)) { AggregatorFactory other = mergedAggregators.get(name); try { - mergedAggregators.put(name, other.getMergingFactory(aggregator)); + // the order of aggregator matters when calling getMergingFactory() + // because it returns a combiningAggregator which can be different from the original aggregator. + mergedAggregators.put(name, aggregator.getMergingFactory(other)); } catch (AggregatorFactoryNotMergeableException ex) { log.warn(ex, "failed to merge aggregator factories"); 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 ff73f2f8c0eb..676647a21767 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; -import java.util.Objects; /** */ @@ -118,29 +117,4 @@ public String toString() ", name='" + name + '\'' + '}'; } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - DoubleMaxAggregatorFactory that = (DoubleMaxAggregatorFactory) o; - - if (!Objects.equals(fieldName, that.fieldName)) { - return false; - } - if (!Objects.equals(expression, that.expression)) { - return false; - } - if (!Objects.equals(name, that.name)) { - return false; - } - - return true; - } } 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 9577e89846c3..eb480a6397d6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; -import java.util.Objects; /** */ @@ -116,29 +115,4 @@ public String toString() ", name='" + name + '\'' + '}'; } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - DoubleMinAggregatorFactory that = (DoubleMinAggregatorFactory) o; - - if (!Objects.equals(fieldName, that.fieldName)) { - return false; - } - if (!Objects.equals(expression, that.expression)) { - return false; - } - if (!Objects.equals(name, that.name)) { - return false; - } - - return true; - } } 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 50ec11bed556..34e398258d38 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; -import java.util.Objects; /** */ @@ -111,30 +110,4 @@ public String toString() ", name='" + name + '\'' + '}'; } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - DoubleSumAggregatorFactory that = (DoubleSumAggregatorFactory) o; - - if (!Objects.equals(fieldName, that.fieldName)) { - return false; - } - if (!Objects.equals(expression, that.expression)) { - return false; - } - if (!Objects.equals(name, that.name)) { - return false; - } - - return true; - } - } diff --git a/processing/src/main/java/io/druid/query/aggregation/SimpleDoubleAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/SimpleDoubleAggregatorFactory.java index e627c126c642..f3b173805f0a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SimpleDoubleAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/SimpleDoubleAggregatorFactory.java @@ -102,6 +102,31 @@ public int hashCode() return Objects.hash(fieldName, expression, name); } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SimpleDoubleAggregatorFactory that = (SimpleDoubleAggregatorFactory) o; + + if (!Objects.equals(fieldName, that.fieldName)) { + return false; + } + if (!Objects.equals(expression, that.expression)) { + return false; + } + if (!Objects.equals(name, that.name)) { + return false; + } + + return true; + } + @Override public Comparator getComparator() { diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index cf82c4d612a3..7deda4745d15 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -31,7 +31,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.NoopAggregator; @@ -200,12 +199,6 @@ public AggregatorFactory getCombiningFactory() return new HyperUniquesAggregatorFactory(name, name, false, round); } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - throw new UnsupportedOperationException("can't merge CardinalityAggregatorFactory"); - } - @Override public List getRequiredColumns() { diff --git a/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java index 561238fb3381..25dc9ec5555b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java @@ -30,7 +30,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -162,16 +161,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { - return getCombiningFactory(); - } else { - throw new AggregatorFactoryNotMergeableException(this, other); - } - } - @Override public List getRequiredColumns() { diff --git a/processing/src/main/java/io/druid/query/aggregation/first/FloatFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/FloatFirstAggregatorFactory.java index 2d896fb042f5..4f4bf2109b9f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/FloatFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/FloatFirstAggregatorFactory.java @@ -30,7 +30,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -160,16 +159,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { - return getCombiningFactory(); - } else { - throw new AggregatorFactoryNotMergeableException(this, other); - } - } - @Override public List getRequiredColumns() { diff --git a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java index cb3ae9cfdb09..0dcd09123ce6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java @@ -29,7 +29,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -153,16 +152,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { - return getCombiningFactory(); - } else { - throw new AggregatorFactoryNotMergeableException(this, other); - } - } - @Override public List getRequiredColumns() { diff --git a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java index a0ed875ecda5..35db2396d42d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java @@ -29,7 +29,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; @@ -153,16 +152,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { - return getCombiningFactory(); - } else { - throw new AggregatorFactoryNotMergeableException(this, other); - } - } - @Override public List getRequiredColumns() { diff --git a/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java index c3a4d10a4bcd..4b2d1df5b520 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java @@ -29,7 +29,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.first.FloatFirstAggregatorFactory; @@ -151,16 +150,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { - return getCombiningFactory(); - } else { - throw new AggregatorFactoryNotMergeableException(this, other); - } - } - @Override public List getRequiredColumns() { diff --git a/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java index 2a3ba885fd24..6e6562b7b16a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java @@ -28,7 +28,6 @@ import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; @@ -149,16 +148,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException - { - if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { - return getCombiningFactory(); - } else { - throw new AggregatorFactoryNotMergeableException(this, other); - } - } - @Override public List getRequiredColumns() { 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 3a75bf8255ee..217a3084646e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -100,7 +100,7 @@ public abstract class IncrementalIndex implements Iterable, // Used to discover ValueType based on the class of values in a row // Also used to convert between the duplicate ValueType enums in DimensionSchema (druid-api) and main druid. - private static final Map TYPE_MAP = ImmutableMap.builder() + public static final Map TYPE_MAP = ImmutableMap.builder() .put(Long.class, ValueType.LONG) .put(Double.class, ValueType.DOUBLE) .put(Float.class, ValueType.FLOAT) From 805c81cd55b174f3c05794ee257e669fa362f17d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Oct 2017 16:50:56 +0900 Subject: [PATCH 10/19] fix test --- api/src/main/java/io/druid/data/input/impl/DimensionSchema.java | 2 +- .../java/io/druid/data/input/impl/NoopInputRowParserTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java index 045dcb776bd1..c37117069285 100644 --- a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java @@ -118,7 +118,7 @@ public static MultiValueHandling ofDefault() protected DimensionSchema(String name, MultiValueHandling multiValueHandling) { this.name = Preconditions.checkNotNull(name, "Dimension name cannot be null."); - this.multiValueHandling = multiValueHandling; + this.multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling; } @JsonProperty diff --git a/api/src/test/java/io/druid/data/input/impl/NoopInputRowParserTest.java b/api/src/test/java/io/druid/data/input/impl/NoopInputRowParserTest.java index d0faf6f963f1..7e3be6947414 100644 --- a/api/src/test/java/io/druid/data/input/impl/NoopInputRowParserTest.java +++ b/api/src/test/java/io/druid/data/input/impl/NoopInputRowParserTest.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; /** From e740743fbf158566c9da586ae94d41021bad3c65 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Oct 2017 16:53:20 +0900 Subject: [PATCH 11/19] fix test --- .../io/druid/query/aggregation/AggregatorFactory.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index 8fab79262b50..ccd88442c817 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -161,9 +161,13 @@ public static AggregatorFactory[] mergeAggregators(List agg if (aggregatorsList.size() == 1) { final AggregatorFactory[] aggregatorFactories = aggregatorsList.get(0); - final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; - Arrays.setAll(combiningFactories, i -> aggregatorFactories[i].getCombiningFactory()); - return combiningFactories; + if (aggregatorFactories != null) { + final AggregatorFactory[] combiningFactories = new AggregatorFactory[aggregatorFactories.length]; + Arrays.setAll(combiningFactories, i -> aggregatorFactories[i].getCombiningFactory()); + return combiningFactories; + } else { + return null; + } } Map mergedAggregators = new LinkedHashMap<>(); From bb64304acec05f238ed7989593c7e6053fb1ee50 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 28 Oct 2017 15:39:47 +0900 Subject: [PATCH 12/19] test for different dimension orderings and types, and doc for type and ordering --- docs/content/ingestion/tasks.md | 13 ++- .../indexing/common/task/CompactionTask.java | 9 ++ .../common/task/CompactionTaskTest.java | 110 ++++++++++-------- 3 files changed, 84 insertions(+), 48 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 847d1c0a4121..1b81c5a8ce68 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -280,8 +280,17 @@ A compaction task internally generates an `index` task spec for performing compa For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html) and `dimensionsSpec` and `metricsSpec` always include all dimensions and metrics of the input segments. -Note that the output segment is rolled up only when `rollup` is set for all input segments. -See [Segment Metadata Queries](../querying/segmentmetadataquery.html#analysistypes) for more details about `rollup`. +The output segment can have different metadata from the input segments unless all input segments have the same metadata. + +- Dimensions: since Druid supports schema change, the dimensions can be different across segments even if they are a part of the same dataSource. +If the input segments have different dimensions, the output segment basically includes all dimensions of the input segments. +Furthermore, even the dimension order or the data type of dimensions can be changed for more optimized performance. For example, the data type of some dimensions can be +changed from `string` to primitive types, or the order of dimensions can be changed for better locality (See [Partitioning](batch-ingestion.html#partitioning-specification)). +In this case, the dimensions of recent segments precede those of older segments in terms of data types and the ordering. +This is because more recent segments are more likely to have the new desired order and data types. +- Roll-up: the output segment is rolled up only when `rollup` is set for all input segments. +See [Roll-up](../design/index.html#roll-up) for more details. +You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.html#analysistypes). Segment Destroying Tasks ------------------------ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index c299dc9353dd..40b2164a1ae2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -171,6 +171,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); } + if (indexTaskSpec.getIngestionSchema() == null) { + log.info("Cannot find segments for interval"); + } + final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(indexTaskSpec); log.info("Generated compaction task details: " + json); @@ -194,6 +198,11 @@ static IndexIngestionSpec createIngestionSchema( ); final Map segmentFileMap = pair.lhs; final List> timelineSegments = pair.rhs; + + if (timelineSegments.size() == 0) { + return null; + } + final DataSchema dataSchema = createDataSchema( dataSource, interval, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index 3c9cb043bbeb..406a37bef2dd 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -23,12 +23,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.impl.DimensionSchema; -import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.data.input.impl.DoubleDimensionSchema; import io.druid.data.input.impl.FloatDimensionSchema; import io.druid.data.input.impl.InputRowParser; @@ -76,7 +73,6 @@ import io.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.Assert; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -96,12 +92,24 @@ public class CompactionTaskTest { private static final String DATA_SOURCE = "dataSource"; private static final String TIMESTAMP_COLUMN = "timestamp"; - private static final Interval INTERVAL = Intervals.of("2017-01-01/2018-01-01"); + private static final String MIXED_TYPE_COLUMN = "string_to_double"; + private static final Interval COMPACTION_INTERVAL = Intervals.of("2017-01-01/2017-06-01"); + private static final Map MIXED_TYPE_COLUMN_MAP = ImmutableMap.of( + Intervals.of("2017-01-01/2017-02-01"), + new StringDimensionSchema(MIXED_TYPE_COLUMN, null), + Intervals.of("2017-02-01/2017-03-01"), + new StringDimensionSchema(MIXED_TYPE_COLUMN, null), + Intervals.of("2017-03-01/2017-04-01"), + new StringDimensionSchema(MIXED_TYPE_COLUMN, null), + Intervals.of("2017-04-01/2017-05-01"), + new StringDimensionSchema(MIXED_TYPE_COLUMN, null), + Intervals.of("2017-05-01/2017-06-01"), + new DoubleDimensionSchema(MIXED_TYPE_COLUMN) + ); private static final IndexTuningConfig TUNING_CONFIG = createTuningConfig(); private static Map DIMENSIONS; private static Map AGGREGATORS; - private static Map SEGMENT_MAP; private static ObjectMapper objectMapper = new DefaultObjectMapper(); private static TaskToolbox toolbox; @@ -116,7 +124,7 @@ public static void setupClass() for (int i = 0; i < 5; i++) { final StringDimensionSchema schema = new StringDimensionSchema( "string_dim_" + i, - MultiValueHandling.SORTED_ARRAY + null ); DIMENSIONS.put(schema.getName(), schema); } @@ -139,17 +147,18 @@ public static void setupClass() AGGREGATORS.put("agg_3", new FloatFirstAggregatorFactory("agg_3", "float_dim_3")); AGGREGATORS.put("agg_4", new DoubleLastAggregatorFactory("agg_4", "double_dim_4")); - SEGMENT_MAP = new HashMap<>(5); + final Map segmentMap = new HashMap<>(5); for (int i = 0; i < 5; i++) { - SEGMENT_MAP.put( + final Interval segmentInterval = Intervals.of(String.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2))); + segmentMap.put( new DataSegment( DATA_SOURCE, - INTERVAL, + segmentInterval, "version", ImmutableMap.of(), - findDimensions(i), + findDimensions(i, segmentInterval), new ArrayList<>(AGGREGATORS.keySet()), - new NumberedShardSpec(i, 5), + new NumberedShardSpec(0, 1), 0, 1 ), @@ -158,22 +167,25 @@ public static void setupClass() } toolbox = new TestTaskToolbox( - new TestTaskActionClient(new ArrayList<>(SEGMENT_MAP.keySet())), - new TestIndexIO(objectMapper, SEGMENT_MAP), - SEGMENT_MAP + new TestTaskActionClient(new ArrayList<>(segmentMap.keySet())), + new TestIndexIO(objectMapper, segmentMap), + segmentMap ); } - private static List findDimensions(int index) + private static List findDimensions(int startIndex, Interval segmentInterval) { final List dimensions = new ArrayList<>(); dimensions.add(TIMESTAMP_COLUMN); - for (int i = index; i < Math.min(index + 3, 5); i++) { - dimensions.add("string_dim_" + i); - dimensions.add("long_dim_" + i); - dimensions.add("float_dim_" + i); - dimensions.add("double_dim_" + i); + for (int i = 0; i < 5; i++) { + int postfix = i + startIndex; + postfix = postfix >= 5 ? postfix - 5 : postfix; + dimensions.add("string_dim_" + postfix); + dimensions.add("long_dim_" + postfix); + dimensions.add("float_dim_" + postfix); + dimensions.add("double_dim_" + postfix); } + dimensions.add(MIXED_TYPE_COLUMN_MAP.get(segmentInterval).getName()); return dimensions; } @@ -200,19 +212,13 @@ private static IndexTuningConfig createTuningConfig() ); } - @Before - public void setup() throws IOException - { - - } - @Test public void testCreateIngestionSchema() throws IOException, SegmentLoadingException { final IndexIngestionSpec ingestionSchema = CompactionTask.createIngestionSchema( toolbox, DATA_SOURCE, - INTERVAL, + COMPACTION_INTERVAL, TUNING_CONFIG, GuiceInjectors.makeStartupInjector(), objectMapper @@ -226,9 +232,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept Assert.assertTrue(parser instanceof NoopInputRowParser); Assert.assertTrue(parser.getParseSpec() instanceof TimeAndDimsParseSpec); Assert.assertEquals( - new HashSet<>(Sets.difference( - new HashSet<>(DIMENSIONS.values()), ImmutableSet.of(new LongDimensionSchema(Column.TIME_COLUMN_NAME))) - ), + findExpectedDimensions(), new HashSet<>(parser.getParseSpec().getDimensionsSpec().getDimensions()) ); final Set expectedAggregators = AGGREGATORS.values() @@ -237,7 +241,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept .collect(Collectors.toSet()); Assert.assertEquals(expectedAggregators, new HashSet<>(Arrays.asList(dataSchema.getAggregators()))); Assert.assertEquals( - new ArbitraryGranularitySpec(Granularities.NONE, false, ImmutableList.of(INTERVAL)), + new ArbitraryGranularitySpec(Granularities.NONE, false, ImmutableList.of(COMPACTION_INTERVAL)), dataSchema.getGranularitySpec() ); @@ -248,7 +252,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept Assert.assertTrue(firehoseFactory instanceof IngestSegmentFirehoseFactory); final IngestSegmentFirehoseFactory ingestSegmentFirehoseFactory = (IngestSegmentFirehoseFactory) firehoseFactory; Assert.assertEquals(DATA_SOURCE, ingestSegmentFirehoseFactory.getDataSource()); - Assert.assertEquals(INTERVAL, ingestSegmentFirehoseFactory.getInterval()); + Assert.assertEquals(COMPACTION_INTERVAL, ingestSegmentFirehoseFactory.getInterval()); Assert.assertNull(ingestSegmentFirehoseFactory.getDimensionsFilter()); // check the order of dimensions Assert.assertEquals( @@ -258,22 +262,23 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept "long_dim_4", "float_dim_4", "double_dim_4", - "string_dim_3", - "long_dim_3", - "float_dim_3", - "double_dim_3", - "string_dim_2", - "long_dim_2", - "float_dim_2", - "double_dim_2", + "string_dim_0", + "long_dim_0", + "float_dim_0", + "double_dim_0", "string_dim_1", "long_dim_1", "float_dim_1", "double_dim_1", - "string_dim_0", - "long_dim_0", - "float_dim_0", - "double_dim_0" + "string_dim_2", + "long_dim_2", + "float_dim_2", + "double_dim_2", + "string_dim_3", + "long_dim_3", + "float_dim_3", + "double_dim_3", + "string_to_double" ), ingestSegmentFirehoseFactory.getDimensions() ); @@ -287,6 +292,17 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept Assert.assertEquals(createTuningConfig(), ingestionSchema.getTuningConfig()); } + private static Set findExpectedDimensions() + { + final Set expectedDimensions = new HashSet<>(); + expectedDimensions.addAll(DIMENSIONS.values()); + // __time column is not included + expectedDimensions.remove(new LongDimensionSchema(Column.TIME_COLUMN_NAME)); + // this column should be double type + expectedDimensions.add(new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + return expectedDimensions; + } + private static class TestTaskToolbox extends TaskToolbox { private final Map segmentFileMap; @@ -380,7 +396,9 @@ private static class TestIndexIO extends IndexIO final List aggregatorFactories = new ArrayList<>(segment.getMetrics().size()); for (String columnName : columnNames) { - if (DIMENSIONS.containsKey(columnName)) { + if (columnName.equals(MIXED_TYPE_COLUMN)) { + columnMap.put(columnName, createColumn(MIXED_TYPE_COLUMN_MAP.get(segment.getInterval()))); + } else if (DIMENSIONS.containsKey(columnName)) { columnMap.put(columnName, createColumn(DIMENSIONS.get(columnName))); } else if (AGGREGATORS.containsKey(columnName)) { columnMap.put(columnName, createColumn(AGGREGATORS.get(columnName))); From f6666c56655e015882a85ca9ff7191b2e1c0e306 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 28 Oct 2017 16:15:45 +0900 Subject: [PATCH 13/19] add control for custom ordering and type --- .../indexing/common/task/CompactionTask.java | 24 ++- .../common/task/CompactionTaskTest.java | 138 ++++++++++++------ 2 files changed, 115 insertions(+), 47 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 40b2164a1ae2..223b94da9d38 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -72,6 +72,7 @@ import io.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -90,6 +91,7 @@ public class CompactionTask extends AbstractTask private static final String TYPE = "compact"; private final Interval interval; + private final DimensionsSpec dimensionsSpec; private final IndexTuningConfig tuningConfig; private final Injector injector; private final ObjectMapper jsonMapper; @@ -103,14 +105,16 @@ public CompactionTask( @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("dataSource") final String dataSource, @JsonProperty("interval") final Interval interval, - @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, - @JsonProperty("context") final Map context, + @Nullable @JsonProperty("dimensions") final DimensionsSpec dimensionsSpec, + @Nullable @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, + @Nullable @JsonProperty("context") final Map context, @JacksonInject Injector injector, @JacksonInject ObjectMapper jsonMapper ) { super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); this.interval = Preconditions.checkNotNull(interval, "interval"); + this.dimensionsSpec = dimensionsSpec; this.tuningConfig = tuningConfig; this.injector = injector; this.jsonMapper = jsonMapper; @@ -122,6 +126,12 @@ public Interval getInterval() return interval; } + @JsonProperty + public DimensionsSpec getDimensionsSpec() + { + return dimensionsSpec; + } + @JsonProperty public IndexTuningConfig getTuningConfig() { @@ -156,6 +166,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception toolbox, getDataSource(), interval, + dimensionsSpec, tuningConfig, injector, jsonMapper @@ -186,6 +197,7 @@ static IndexIngestionSpec createIngestionSchema( TaskToolbox toolbox, String dataSource, Interval interval, + DimensionsSpec dimensionsSpec, IndexTuningConfig tuningConfig, Injector injector, ObjectMapper jsonMapper @@ -206,6 +218,7 @@ static IndexIngestionSpec createIngestionSchema( final DataSchema dataSchema = createDataSchema( dataSource, interval, + dimensionsSpec, toolbox.getIndexIO(), jsonMapper, timelineSegments, @@ -248,6 +261,7 @@ private static Pair, List> timelineSegments, @@ -279,8 +293,10 @@ private static DataSchema createDataSchema( ); // find unique dimensions - final DimensionsSpec dimensionsSpec = createDimensionsSpec(queryableIndices); - final InputRowParser parser = new NoopInputRowParser(new TimeAndDimsParseSpec(null, dimensionsSpec)); + final DimensionsSpec finalDimensionsSpec = dimensionsSpec == null ? + createDimensionsSpec(queryableIndices) : + dimensionsSpec; + final InputRowParser parser = new NoopInputRowParser(new TimeAndDimsParseSpec(null, finalDimensionsSpec)); return new DataSchema( dataSource, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index 406a37bef2dd..49cf31adeadf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.DoubleDimensionSchema; import io.druid.data.input.impl.FloatDimensionSchema; import io.druid.data.input.impl.InputRowParser; @@ -75,11 +76,15 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -88,6 +93,7 @@ import java.util.Set; import java.util.stream.Collectors; +@RunWith(Parameterized.class) public class CompactionTaskTest { private static final String DATA_SOURCE = "dataSource"; @@ -114,7 +120,7 @@ public class CompactionTaskTest private static TaskToolbox toolbox; @BeforeClass - public static void setupClass() + public static void setup() { DIMENSIONS = new HashMap<>(); AGGREGATORS = new HashMap<>(); @@ -212,18 +218,100 @@ private static IndexTuningConfig createTuningConfig() ); } - @Test - public void testCreateIngestionSchema() throws IOException, SegmentLoadingException + @Parameters(name = "{0}") + public static Collection constructorFeeder() { - final IndexIngestionSpec ingestionSchema = CompactionTask.createIngestionSchema( + final DimensionsSpec customSpec = new DimensionsSpec( + Lists.newArrayList( + new LongDimensionSchema("timestamp"), + new StringDimensionSchema("string_dim_0"), + new StringDimensionSchema("string_dim_1"), + new StringDimensionSchema("string_dim_2"), + new StringDimensionSchema("string_dim_3"), + new StringDimensionSchema("string_dim_4"), + new LongDimensionSchema("long_dim_0"), + new LongDimensionSchema("long_dim_1"), + new LongDimensionSchema("long_dim_2"), + new LongDimensionSchema("long_dim_3"), + new LongDimensionSchema("long_dim_4"), + new FloatDimensionSchema("float_dim_0"), + new FloatDimensionSchema("float_dim_1"), + new FloatDimensionSchema("float_dim_2"), + new FloatDimensionSchema("float_dim_3"), + new FloatDimensionSchema("float_dim_4"), + new DoubleDimensionSchema("double_dim_0"), + new DoubleDimensionSchema("double_dim_1"), + new DoubleDimensionSchema("double_dim_2"), + new DoubleDimensionSchema("double_dim_3"), + new DoubleDimensionSchema("double_dim_4"), + new StringDimensionSchema(MIXED_TYPE_COLUMN) + ), + null, + null + ); + + return ImmutableList.of( + new Object[]{ + "autoDimensionsSpec", + null, + new DimensionsSpec( + Lists.newArrayList( + new LongDimensionSchema("timestamp"), + new StringDimensionSchema("string_dim_4"), + new LongDimensionSchema("long_dim_4"), + new FloatDimensionSchema("float_dim_4"), + new DoubleDimensionSchema("double_dim_4"), + new StringDimensionSchema("string_dim_0"), + new LongDimensionSchema("long_dim_0"), + new FloatDimensionSchema("float_dim_0"), + new DoubleDimensionSchema("double_dim_0"), + new StringDimensionSchema("string_dim_1"), + new LongDimensionSchema("long_dim_1"), + new FloatDimensionSchema("float_dim_1"), + new DoubleDimensionSchema("double_dim_1"), + new StringDimensionSchema("string_dim_2"), + new LongDimensionSchema("long_dim_2"), + new FloatDimensionSchema("float_dim_2"), + new DoubleDimensionSchema("double_dim_2"), + new StringDimensionSchema("string_dim_3"), + new LongDimensionSchema("long_dim_3"), + new FloatDimensionSchema("float_dim_3"), + new DoubleDimensionSchema("double_dim_3"), + new DoubleDimensionSchema("string_to_double") + ), + null, + null + ) + }, + new Object[]{ + "customDimensionsSpec", + customSpec, + customSpec + } + ); + } + + private final IndexIngestionSpec ingestionSchema; + private final DimensionsSpec expectedDimensionsSpec; + + public CompactionTaskTest(String testName, DimensionsSpec dimensionsSpec, DimensionsSpec expectedDimensionsSpec) + throws IOException, SegmentLoadingException + { + this.ingestionSchema = CompactionTask.createIngestionSchema( toolbox, DATA_SOURCE, COMPACTION_INTERVAL, + dimensionsSpec, TUNING_CONFIG, GuiceInjectors.makeStartupInjector(), objectMapper ); + this.expectedDimensionsSpec = expectedDimensionsSpec; + } + @Test + public void testCreateIngestionSchema() throws IOException, SegmentLoadingException + { // assert dataSchema final DataSchema dataSchema = ingestionSchema.getDataSchema(); Assert.assertEquals(DATA_SOURCE, dataSchema.getDataSource()); @@ -232,7 +320,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept Assert.assertTrue(parser instanceof NoopInputRowParser); Assert.assertTrue(parser.getParseSpec() instanceof TimeAndDimsParseSpec); Assert.assertEquals( - findExpectedDimensions(), + new HashSet<>(expectedDimensionsSpec.getDimensions()), new HashSet<>(parser.getParseSpec().getDimensionsSpec().getDimensions()) ); final Set expectedAggregators = AGGREGATORS.values() @@ -254,34 +342,9 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept Assert.assertEquals(DATA_SOURCE, ingestSegmentFirehoseFactory.getDataSource()); Assert.assertEquals(COMPACTION_INTERVAL, ingestSegmentFirehoseFactory.getInterval()); Assert.assertNull(ingestSegmentFirehoseFactory.getDimensionsFilter()); + // check the order of dimensions - Assert.assertEquals( - Lists.newArrayList( - "timestamp", - "string_dim_4", - "long_dim_4", - "float_dim_4", - "double_dim_4", - "string_dim_0", - "long_dim_0", - "float_dim_0", - "double_dim_0", - "string_dim_1", - "long_dim_1", - "float_dim_1", - "double_dim_1", - "string_dim_2", - "long_dim_2", - "float_dim_2", - "double_dim_2", - "string_dim_3", - "long_dim_3", - "float_dim_3", - "double_dim_3", - "string_to_double" - ), - ingestSegmentFirehoseFactory.getDimensions() - ); + Assert.assertEquals(expectedDimensionsSpec.getDimensionNames(), ingestSegmentFirehoseFactory.getDimensions()); // check the order of metrics Assert.assertEquals( Lists.newArrayList("agg_4", "agg_3", "agg_2", "agg_1", "agg_0"), @@ -292,17 +355,6 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept Assert.assertEquals(createTuningConfig(), ingestionSchema.getTuningConfig()); } - private static Set findExpectedDimensions() - { - final Set expectedDimensions = new HashSet<>(); - expectedDimensions.addAll(DIMENSIONS.values()); - // __time column is not included - expectedDimensions.remove(new LongDimensionSchema(Column.TIME_COLUMN_NAME)); - // this column should be double type - expectedDimensions.add(new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); - return expectedDimensions; - } - private static class TestTaskToolbox extends TaskToolbox { private final Map segmentFileMap; From 3e1f5a3234c6ed97f5ac0f5c26ae65fd040b4ca8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 31 Oct 2017 09:40:35 +0900 Subject: [PATCH 14/19] update doc --- docs/content/ingestion/tasks.md | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 1b81c5a8ce68..628e2c9b9762 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -249,6 +249,7 @@ Compaction tasks merge all segments of the given interval. The syntax is: "id": , "dataSource": , "interval": , + "dimensions" , "tuningConfig" , "context": } @@ -260,6 +261,7 @@ Compaction tasks merge all segments of the given interval. The syntax is: |`id`|Task id|No| |`dataSource`|dataSource name to be compacted|Yes| |`interval`|interval of segments to be compacted|Yes| +|`dimensions`|custom dimensionsSpec. compaction task will use this dimensionsSpec if exist instead of generating one. See below for more details.|No| |`tuningConfig`|[Index task tuningConfig](#tuningconfig)|No| |`context`|[Task context](#taskcontext)|No| @@ -277,17 +279,18 @@ This compaction task merges _all segments_ of the interval `2017-01-01/2018-01-0 To merge each day's worth of data into a separate segment, you can submit multiple `compact` tasks, one for each day. They will run in parallel. A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters. -For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html) and `dimensionsSpec` and `metricsSpec` -always include all dimensions and metrics of the input segments. +For example, its `firehose` is always the [ingestSegmentSpec](./firehose.html), and `dimensionsSpec` and `metricsSpec` +include all dimensions and metrics of the input segments by default. The output segment can have different metadata from the input segments unless all input segments have the same metadata. - Dimensions: since Druid supports schema change, the dimensions can be different across segments even if they are a part of the same dataSource. If the input segments have different dimensions, the output segment basically includes all dimensions of the input segments. -Furthermore, even the dimension order or the data type of dimensions can be changed for more optimized performance. For example, the data type of some dimensions can be +However, even if the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. For example, the data type of some dimensions can be changed from `string` to primitive types, or the order of dimensions can be changed for better locality (See [Partitioning](batch-ingestion.html#partitioning-specification)). -In this case, the dimensions of recent segments precede those of older segments in terms of data types and the ordering. -This is because more recent segments are more likely to have the new desired order and data types. +In this case, the dimensions of recent segments precede that of old segments in terms of data types and the ordering. +This is because more recent segments are more likely to have the new desired order and data types. If you want to use +your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec. - Roll-up: the output segment is rolled up only when `rollup` is set for all input segments. See [Roll-up](../design/index.html#roll-up) for more details. You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.html#analysistypes). From 02ff1bdba0bca3e9b36e8431bd87c183994bb09c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 31 Oct 2017 11:27:34 +0900 Subject: [PATCH 15/19] fix compile --- .../java/io/druid/indexing/common/task/CompactionTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index 58381e470129..fc91229e7333 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -319,7 +319,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept final InputRowParser parser = objectMapper.convertValue(dataSchema.getParser(), InputRowParser.class); Assert.assertTrue(parser instanceof TransformingInputRowParser); - Assert.assertTrue(((TransformingInputRowParser)parser).getParser() instanceof NoopInputRowParser); + Assert.assertTrue(((TransformingInputRowParser) parser).getParser() instanceof NoopInputRowParser); Assert.assertTrue(parser.getParseSpec() instanceof TimeAndDimsParseSpec); Assert.assertEquals( new HashSet<>(expectedDimensionsSpec.getDimensions()), From d540f75fc1dc76b30d8fc5b97bcdcb31715f6c0b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 31 Oct 2017 13:08:14 +0900 Subject: [PATCH 16/19] fix compile --- .../java/io/druid/indexing/common/task/CompactionTaskTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index fc91229e7333..1d38fd09067b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -46,6 +46,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -156,7 +157,7 @@ public static void setup() final Map segmentMap = new HashMap<>(5); for (int i = 0; i < 5; i++) { - final Interval segmentInterval = Intervals.of(String.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2))); + final Interval segmentInterval = Intervals.of(StringUtils.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2))); segmentMap.put( new DataSegment( DATA_SOURCE, From 58109d3bd72e173c2ed24ea94f0774959e5256ee Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 31 Oct 2017 18:14:14 +0900 Subject: [PATCH 17/19] add segments param --- .../indexing/common/task/CompactionTask.java | 94 ++++++++--- .../io/druid/indexing/common/task/Tasks.java | 9 -- .../common/task/CompactionTaskTest.java | 153 +++++++++++------- .../io/druid/java/util/common/JodaUtils.java | 14 +- 4 files changed, 181 insertions(+), 89 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index bb4a25254d0e..34981aa533a0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -50,6 +50,7 @@ import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.NoneGranularity; import io.druid.java.util.common.guava.Comparators; @@ -77,6 +78,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -90,8 +92,8 @@ public class CompactionTask extends AbstractTask private static final Logger log = new Logger(CompactionTask.class); private static final String TYPE = "compact"; - private final Interval interval; private final DimensionsSpec dimensionsSpec; + private final SegmentProvider segmentProvider; private final IndexTuningConfig tuningConfig; private final Injector injector; private final ObjectMapper jsonMapper; @@ -104,7 +106,8 @@ public CompactionTask( @JsonProperty("id") final String id, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("dataSource") final String dataSource, - @JsonProperty("interval") final Interval interval, + @Nullable @JsonProperty("interval") final Interval interval, + @Nullable @JsonProperty("segments") final List segments, @Nullable @JsonProperty("dimensions") final DimensionsSpec dimensionsSpec, @Nullable @JsonProperty("tuningConfig") final IndexTuningConfig tuningConfig, @Nullable @JsonProperty("context") final Map context, @@ -113,7 +116,10 @@ public CompactionTask( ) { super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); - this.interval = Preconditions.checkNotNull(interval, "interval"); + Preconditions.checkArgument(interval != null || segments != null, "interval or segments should be specified"); + Preconditions.checkArgument(interval == null || segments == null, "one of interval and segments should be null"); + + this.segmentProvider = segments == null ? new SegmentProvider(dataSource, interval) : new SegmentProvider(segments); this.dimensionsSpec = dimensionsSpec; this.tuningConfig = tuningConfig; this.injector = injector; @@ -123,7 +129,13 @@ public CompactionTask( @JsonProperty public Interval getInterval() { - return interval; + return segmentProvider.interval; + } + + @JsonProperty + public List getSegments() + { + return segmentProvider.segments; } @JsonProperty @@ -154,7 +166,7 @@ public int getPriority() public boolean isReady(TaskActionClient taskActionClient) throws Exception { final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - intervals.add(interval); + intervals.add(segmentProvider.interval); return IndexTask.isReady(taskActionClient, intervals); } @@ -164,8 +176,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception if (indexTaskSpec == null) { final IndexIngestionSpec ingestionSpec = createIngestionSchema( toolbox, - getDataSource(), - interval, + segmentProvider, dimensionsSpec, tuningConfig, injector, @@ -195,8 +206,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception @VisibleForTesting static IndexIngestionSpec createIngestionSchema( TaskToolbox toolbox, - String dataSource, - Interval interval, + SegmentProvider segmentProvider, DimensionsSpec dimensionsSpec, IndexTuningConfig tuningConfig, Injector injector, @@ -205,8 +215,7 @@ static IndexIngestionSpec createIngestionSchema( { Pair, List>> pair = prepareSegments( toolbox, - dataSource, - interval + segmentProvider ); final Map segmentFileMap = pair.lhs; final List> timelineSegments = pair.rhs; @@ -216,8 +225,8 @@ static IndexIngestionSpec createIngestionSchema( } final DataSchema dataSchema = createDataSchema( - dataSource, - interval, + segmentProvider.dataSource, + segmentProvider.interval, dimensionsSpec, toolbox.getIndexIO(), jsonMapper, @@ -228,8 +237,8 @@ static IndexIngestionSpec createIngestionSchema( dataSchema, new IndexIOConfig( new IngestSegmentFirehoseFactory( - dataSource, - interval, + segmentProvider.dataSource, + segmentProvider.interval, null, // no filter // set dimensions and metrics names to make sure that the generated dataSchema is used for the firehose dataSchema.getParser().getParseSpec().getDimensionsSpec().getDimensionNames(), @@ -245,16 +254,14 @@ static IndexIngestionSpec createIngestionSchema( private static Pair, List>> prepareSegments( TaskToolbox toolbox, - String dataSource, - Interval interval + SegmentProvider segmentProvider ) throws IOException, SegmentLoadingException { - final List usedSegments = toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(dataSource, interval, null)); + final List usedSegments = segmentProvider.checkAndGetSegments(toolbox); final Map segmentFileMap = toolbox.fetchSegments(usedSegments); final List> timelineSegments = VersionedIntervalTimeline .forSegments(usedSegments) - .lookup(interval); + .lookup(segmentProvider.interval); return Pair.of(segmentFileMap, timelineSegments); } @@ -422,4 +429,51 @@ private static DimensionSchema createDimensionSchema( throw new ISE("Unsupported value type[%s] for dimension[%s]", type, name); } } + + @VisibleForTesting + static class SegmentProvider + { + private final String dataSource; + private final Interval interval; + private final List segments; + + SegmentProvider(String dataSource, Interval interval) + { + this.dataSource = Preconditions.checkNotNull(dataSource); + this.interval = Preconditions.checkNotNull(interval); + this.segments = null; + } + + SegmentProvider(List segments) + { + Preconditions.checkArgument(segments != null && !segments.isEmpty()); + final String dataSource = segments.get(0).getDataSource(); + Preconditions.checkArgument( + segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)), + "segments should have the same dataSource" + ); + this.segments = segments; + this.dataSource = dataSource; + this.interval = JodaUtils.umbrellaInterval( + segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + } + + List checkAndGetSegments(TaskToolbox toolbox) throws IOException + { + final List usedSegments = toolbox.getTaskActionClient() + .submit(new SegmentListUsedAction(dataSource, interval, null)); + if (segments != null) { + Collections.sort(usedSegments); + Collections.sort(segments); + Preconditions.checkState( + usedSegments.equals(segments), + "Specified segments[%s] are different from the currently used segments[%s]", + segments, + usedSegments + ); + } + return usedSegments; + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java index 6eb36405d7b5..6cceec285d01 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Tasks.java @@ -24,8 +24,6 @@ import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.overlord.LockResult; -import io.druid.java.util.common.ISE; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.guava.Comparators; import org.joda.time.Interval; @@ -49,13 +47,6 @@ public class Tasks public static final String PRIORITY_KEY = "priority"; public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; - public static void checkLockResult(LockResult result, Interval interval) - { - if (!result.isOk()) { - throw new ISE("Failed to lock for interval[%s]", interval); - } - } - public static Map tryAcquireExclusiveLocks(TaskActionClient client, SortedSet intervals) throws IOException { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index 1d38fd09067b..d56779127695 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -39,6 +39,7 @@ import io.druid.indexing.common.actions.SegmentListUsedAction; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.CompactionTask.SegmentProvider; import io.druid.indexing.common.task.IndexTask.IndexIOConfig; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; @@ -74,19 +75,18 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; +import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.rules.ExpectedException; import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -95,7 +95,6 @@ import java.util.Set; import java.util.stream.Collectors; -@RunWith(Parameterized.class) public class CompactionTaskTest { private static final String DATA_SOURCE = "dataSource"; @@ -118,6 +117,7 @@ public class CompactionTaskTest private static Map DIMENSIONS; private static Map AGGREGATORS; + private static List SEGMENTS; private static ObjectMapper objectMapper = new DefaultObjectMapper(); private static TaskToolbox toolbox; @@ -173,6 +173,7 @@ public static void setup() new File("file_" + i) ); } + SEGMENTS = new ArrayList<>(segmentMap.keySet()); toolbox = new TestTaskToolbox( new TestTaskActionClient(new ArrayList<>(segmentMap.keySet())), @@ -220,8 +221,27 @@ private static IndexTuningConfig createTuningConfig() ); } - @Parameters(name = "{0}") - public static Collection constructorFeeder() + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testCreateIngestionSchema() throws IOException, SegmentLoadingException + { + final IndexIngestionSpec ingestionSchema = CompactionTask.createIngestionSchema( + toolbox, + new SegmentProvider(DATA_SOURCE, COMPACTION_INTERVAL), + null, + TUNING_CONFIG, + GuiceInjectors.makeStartupInjector(), + objectMapper + ); + final DimensionsSpec expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); + + assertIngestionSchema(ingestionSchema, expectedDimensionsSpec); + } + + @Test + public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOException, SegmentLoadingException { final DimensionsSpec customSpec = new DimensionsSpec( Lists.newArrayList( @@ -252,67 +272,88 @@ public static Collection constructorFeeder() null ); - return ImmutableList.of( - new Object[]{ - "autoDimensionsSpec", - null, - new DimensionsSpec( - Lists.newArrayList( - new LongDimensionSchema("timestamp"), - new StringDimensionSchema("string_dim_4"), - new LongDimensionSchema("long_dim_4"), - new FloatDimensionSchema("float_dim_4"), - new DoubleDimensionSchema("double_dim_4"), - new StringDimensionSchema("string_dim_0"), - new LongDimensionSchema("long_dim_0"), - new FloatDimensionSchema("float_dim_0"), - new DoubleDimensionSchema("double_dim_0"), - new StringDimensionSchema("string_dim_1"), - new LongDimensionSchema("long_dim_1"), - new FloatDimensionSchema("float_dim_1"), - new DoubleDimensionSchema("double_dim_1"), - new StringDimensionSchema("string_dim_2"), - new LongDimensionSchema("long_dim_2"), - new FloatDimensionSchema("float_dim_2"), - new DoubleDimensionSchema("double_dim_2"), - new StringDimensionSchema("string_dim_3"), - new LongDimensionSchema("long_dim_3"), - new FloatDimensionSchema("float_dim_3"), - new DoubleDimensionSchema("double_dim_3"), - new DoubleDimensionSchema("string_to_double") - ), - null, - null - ) - }, - new Object[]{ - "customDimensionsSpec", - customSpec, - customSpec - } + final IndexIngestionSpec ingestionSchema = CompactionTask.createIngestionSchema( + toolbox, + new SegmentProvider(DATA_SOURCE, COMPACTION_INTERVAL), + customSpec, + TUNING_CONFIG, + GuiceInjectors.makeStartupInjector(), + objectMapper ); - } - private final IndexIngestionSpec ingestionSchema; - private final DimensionsSpec expectedDimensionsSpec; + assertIngestionSchema(ingestionSchema, customSpec); + } - public CompactionTaskTest(String testName, DimensionsSpec dimensionsSpec, DimensionsSpec expectedDimensionsSpec) - throws IOException, SegmentLoadingException + @Test + public void testCreateIngestionSchemaWithCustomSegments() throws IOException, SegmentLoadingException { - this.ingestionSchema = CompactionTask.createIngestionSchema( + final IndexIngestionSpec ingestionSchema = CompactionTask.createIngestionSchema( toolbox, - DATA_SOURCE, - COMPACTION_INTERVAL, - dimensionsSpec, + new SegmentProvider(SEGMENTS), + null, TUNING_CONFIG, GuiceInjectors.makeStartupInjector(), objectMapper ); - this.expectedDimensionsSpec = expectedDimensionsSpec; + final DimensionsSpec expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); + + assertIngestionSchema(ingestionSchema, expectedDimensionsSpec); } @Test - public void testCreateIngestionSchema() throws IOException, SegmentLoadingException + public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOException, SegmentLoadingException + { + expectedException.expect(CoreMatchers.instanceOf(IllegalStateException.class)); + expectedException.expectMessage(CoreMatchers.containsString("are different from the currently used segments")); + + final List segments = new ArrayList<>(SEGMENTS); + segments.remove(0); + CompactionTask.createIngestionSchema( + toolbox, + new SegmentProvider(segments), + null, + TUNING_CONFIG, + GuiceInjectors.makeStartupInjector(), + objectMapper + ); + } + + private static DimensionsSpec getExpectedDimensionsSpecForAutoGeneration() + { + return new DimensionsSpec( + Lists.newArrayList( + new LongDimensionSchema("timestamp"), + new StringDimensionSchema("string_dim_4"), + new LongDimensionSchema("long_dim_4"), + new FloatDimensionSchema("float_dim_4"), + new DoubleDimensionSchema("double_dim_4"), + new StringDimensionSchema("string_dim_0"), + new LongDimensionSchema("long_dim_0"), + new FloatDimensionSchema("float_dim_0"), + new DoubleDimensionSchema("double_dim_0"), + new StringDimensionSchema("string_dim_1"), + new LongDimensionSchema("long_dim_1"), + new FloatDimensionSchema("float_dim_1"), + new DoubleDimensionSchema("double_dim_1"), + new StringDimensionSchema("string_dim_2"), + new LongDimensionSchema("long_dim_2"), + new FloatDimensionSchema("float_dim_2"), + new DoubleDimensionSchema("double_dim_2"), + new StringDimensionSchema("string_dim_3"), + new LongDimensionSchema("long_dim_3"), + new FloatDimensionSchema("float_dim_3"), + new DoubleDimensionSchema("double_dim_3"), + new DoubleDimensionSchema("string_to_double") + ), + null, + null + ); + } + + private static void assertIngestionSchema( + IndexIngestionSpec ingestionSchema, + DimensionsSpec expectedDimensionsSpec + ) { // assert dataSchema final DataSchema dataSchema = ingestionSchema.getDataSchema(); diff --git a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java index bee564ba3f62..a557ea7d442d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java @@ -28,7 +28,7 @@ import java.util.ArrayList; import java.util.Iterator; -import java.util.TreeSet; +import java.util.SortedSet; /** */ @@ -42,9 +42,15 @@ public static ArrayList condenseIntervals(Iterable intervals { ArrayList retVal = Lists.newArrayList(); - TreeSet sortedIntervals = Sets.newTreeSet(Comparators.intervalsByStartThenEnd()); - for (Interval interval : intervals) { - sortedIntervals.add(interval); + final SortedSet sortedIntervals; + + if (intervals instanceof SortedSet) { + sortedIntervals = (SortedSet) intervals; + } else { + sortedIntervals = Sets.newTreeSet(Comparators.intervalsByStartThenEnd()); + for (Interval interval : intervals) { + sortedIntervals.add(interval); + } } if (sortedIntervals.isEmpty()) { From f9e5e0631874aab933f265556fd0d90ec64d7063 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 1 Nov 2017 13:20:04 +0900 Subject: [PATCH 18/19] fix serde error --- .../indexing/common/task/CompactionTask.java | 24 ++++- .../common/task/CompactionTaskTest.java | 88 +++++++++++++++++-- 2 files changed, 103 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 34981aa533a0..768aa4c285ee 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -92,11 +92,14 @@ public class CompactionTask extends AbstractTask private static final Logger log = new Logger(CompactionTask.class); private static final String TYPE = "compact"; + private final Interval interval; + private final List segments; private final DimensionsSpec dimensionsSpec; - private final SegmentProvider segmentProvider; private final IndexTuningConfig tuningConfig; private final Injector injector; private final ObjectMapper jsonMapper; + @JsonIgnore + private final SegmentProvider segmentProvider; @JsonIgnore private IndexTask indexTaskSpec; @@ -119,23 +122,25 @@ public CompactionTask( Preconditions.checkArgument(interval != null || segments != null, "interval or segments should be specified"); Preconditions.checkArgument(interval == null || segments == null, "one of interval and segments should be null"); - this.segmentProvider = segments == null ? new SegmentProvider(dataSource, interval) : new SegmentProvider(segments); + this.interval = interval; + this.segments = segments; this.dimensionsSpec = dimensionsSpec; this.tuningConfig = tuningConfig; this.injector = injector; this.jsonMapper = jsonMapper; + this.segmentProvider = segments == null ? new SegmentProvider(dataSource, interval) : new SegmentProvider(segments); } @JsonProperty public Interval getInterval() { - return segmentProvider.interval; + return interval; } @JsonProperty public List getSegments() { - return segmentProvider.segments; + return segments; } @JsonProperty @@ -162,6 +167,12 @@ public int getPriority() return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_MERGE_TASK_PRIORITY); } + @VisibleForTesting + SegmentProvider getSegmentProvider() + { + return segmentProvider; + } + @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { @@ -459,6 +470,11 @@ static class SegmentProvider ); } + List getSegments() + { + return segments; + } + List checkAndGetSegments(TaskToolbox toolbox) throws IOException { final List usedSegments = toolbox.getTaskActionClient() diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index d56779127695..84bb50572aba 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -20,10 +20,14 @@ package io.druid.indexing.common.task; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.inject.Injector; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; @@ -34,6 +38,8 @@ import io.druid.data.input.impl.NoopInputRowParser; import io.druid.data.input.impl.StringDimensionSchema; import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.guice.GuiceAnnotationIntrospector; +import io.druid.guice.GuiceInjectableValues; import io.druid.guice.GuiceInjectors; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; @@ -114,11 +120,12 @@ public class CompactionTaskTest new DoubleDimensionSchema(MIXED_TYPE_COLUMN) ); private static final IndexTuningConfig TUNING_CONFIG = createTuningConfig(); + private static final Injector INJECTOR = GuiceInjectors.makeStartupInjector(); private static Map DIMENSIONS; private static Map AGGREGATORS; private static List SEGMENTS; - private static ObjectMapper objectMapper = new DefaultObjectMapper(); + private static ObjectMapper objectMapper = setupInjectablesInObjectMapper(new DefaultObjectMapper()); private static TaskToolbox toolbox; @BeforeClass @@ -182,6 +189,24 @@ public static void setup() ); } + private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper) + { + final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); + objectMapper.setAnnotationIntrospectors( + new AnnotationIntrospectorPair( + guiceIntrospector, objectMapper.getSerializationConfig().getAnnotationIntrospector() + ), + new AnnotationIntrospectorPair( + guiceIntrospector, objectMapper.getDeserializationConfig().getAnnotationIntrospector() + ) + ); + objectMapper.setInjectableValues(new GuiceInjectableValues(INJECTOR)); + objectMapper.registerModule( + new SimpleModule().registerSubtypes(new NamedType(NumberedShardSpec.class, "NumberedShardSpec")) + ); + return objectMapper; + } + private static List findDimensions(int startIndex, Interval segmentInterval) { final List dimensions = new ArrayList<>(); @@ -224,6 +249,59 @@ private static IndexTuningConfig createTuningConfig() @Rule public ExpectedException expectedException = ExpectedException.none(); + @Test + public void testSerdeWithInterval() throws IOException + { + final CompactionTask task = new CompactionTask( + null, + null, + DATA_SOURCE, + COMPACTION_INTERVAL, + null, + null, + createTuningConfig(), + ImmutableMap.of("testKey", "testContext"), + INJECTOR, + objectMapper + ); + final byte[] bytes = objectMapper.writeValueAsBytes(task); + final CompactionTask fromJson = objectMapper.readValue(bytes, CompactionTask.class); + Assert.assertEquals(task.getType(), fromJson.getType()); + Assert.assertEquals(task.getDataSource(), fromJson.getDataSource()); + Assert.assertEquals(task.getInterval(), fromJson.getInterval()); + Assert.assertEquals(task.getSegments(), fromJson.getSegments()); + Assert.assertEquals(task.getDimensionsSpec(), fromJson.getDimensionsSpec()); + Assert.assertEquals(task.getTuningConfig(), fromJson.getTuningConfig()); + Assert.assertEquals(task.getContext(), fromJson.getContext()); + Assert.assertNull(fromJson.getSegmentProvider().getSegments()); + } + + @Test + public void testSerdeWithSegments() throws IOException + { + final CompactionTask task = new CompactionTask( + null, + null, + DATA_SOURCE, + null, + SEGMENTS, + null, + createTuningConfig(), + ImmutableMap.of("testKey", "testContext"), + INJECTOR, + objectMapper + ); + final byte[] bytes = objectMapper.writeValueAsBytes(task); + final CompactionTask fromJson = objectMapper.readValue(bytes, CompactionTask.class); + Assert.assertEquals(task.getType(), fromJson.getType()); + Assert.assertEquals(task.getDataSource(), fromJson.getDataSource()); + Assert.assertEquals(task.getInterval(), fromJson.getInterval()); + Assert.assertEquals(task.getSegments(), fromJson.getSegments()); + Assert.assertEquals(task.getDimensionsSpec(), fromJson.getDimensionsSpec()); + Assert.assertEquals(task.getTuningConfig(), fromJson.getTuningConfig()); + Assert.assertEquals(task.getContext(), fromJson.getContext()); + } + @Test public void testCreateIngestionSchema() throws IOException, SegmentLoadingException { @@ -232,7 +310,7 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept new SegmentProvider(DATA_SOURCE, COMPACTION_INTERVAL), null, TUNING_CONFIG, - GuiceInjectors.makeStartupInjector(), + INJECTOR, objectMapper ); final DimensionsSpec expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); @@ -277,7 +355,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti new SegmentProvider(DATA_SOURCE, COMPACTION_INTERVAL), customSpec, TUNING_CONFIG, - GuiceInjectors.makeStartupInjector(), + INJECTOR, objectMapper ); @@ -292,7 +370,7 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException, Se new SegmentProvider(SEGMENTS), null, TUNING_CONFIG, - GuiceInjectors.makeStartupInjector(), + INJECTOR, objectMapper ); final DimensionsSpec expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); @@ -313,7 +391,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio new SegmentProvider(segments), null, TUNING_CONFIG, - GuiceInjectors.makeStartupInjector(), + INJECTOR, objectMapper ); } From 95a8a71879c62da86b94a8c3d4c93a19674941ab Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 3 Nov 2017 16:00:58 +0900 Subject: [PATCH 19/19] fix build --- .../java/io/druid/indexing/common/task/CompactionTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index 84bb50572aba..4bca5b532258 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -76,9 +76,9 @@ import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.TransformingInputRowParser; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.loading.SegmentLoadingException; +import io.druid.segment.transform.TransformingInputRowParser; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.hamcrest.CoreMatchers;