From b9b072b05fd6af3af9f1fff748b22af4e448d4f9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 15 Sep 2015 13:31:14 -0700 Subject: [PATCH 1/7] Nix Committers.supplierOf; Suppliers.ofInstance is good enough. --- .../segment/realtime/plumber/Committers.java | 42 +++++++------------ .../plumber/RealtimePlumberSchoolTest.java | 2 +- 2 files changed, 16 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Committers.java b/server/src/main/java/io/druid/segment/realtime/plumber/Committers.java index b3089c592faa..7659511369c8 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Committers.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Committers.java @@ -20,6 +20,7 @@ package io.druid.segment.realtime.plumber; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseV2; @@ -43,22 +44,21 @@ public void run() public static Supplier supplierFromRunnable(final Runnable runnable) { - return supplierOf( - new Committer() - { - @Override - public Object getMetadata() - { - return null; - } + final Committer committer = new Committer() + { + @Override + public Object getMetadata() + { + return null; + } - @Override - public void run() - { - runnable.run(); - } - } - ); + @Override + public void run() + { + runnable.run(); + } + }; + return Suppliers.ofInstance(committer); } public static Supplier supplierFromFirehose(final Firehose firehose) @@ -103,16 +103,4 @@ public static Committer nil() { return NIL; } - - public static Supplier supplierOf(final Committer committer) - { - return new Supplier() - { - @Override - public Committer get() - { - return committer; - } - }; - } } diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 1f60c8cd7a5e..e2eeb44c5f5b 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -295,7 +295,7 @@ public void testPersistFails() throws Exception EasyMock.expect(row.getTimestampFromEpoch()).andReturn(0L); EasyMock.expect(row.getDimensions()).andReturn(new ArrayList()); EasyMock.replay(row); - plumber.add(row, Committers.supplierOf(Committers.nil())); + plumber.add(row, Suppliers.ofInstance(Committers.nil())); plumber.persist( Committers.supplierFromRunnable( new Runnable() From 8d1a285160e2ba331c8914e9b0f28802ba6785bd Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 14 Aug 2015 10:00:15 -0700 Subject: [PATCH 2/7] Appenderatoring. Appenderators are a way of getting more control over the ingestion process than a Plumber allows. The idea is that existing Plumbers could be implemented using Appenderators, but you could also implement things that Plumbers can't do. Also: - RealtimeTuningConfig: Default basePersistDirectory now different per instance. - Sink: Ability to consider itself "finished" vs "still writable". --- .../indexing/RealtimeTuningConfig.java | 25 +- .../realtime/appenderator/Appenderator.java | 171 +++ .../appenderator/AppenderatorImpl.java | 1158 +++++++++++++++++ .../realtime/appenderator/Appenderators.java | 118 ++ .../realtime/appenderator/Committed.java | 119 ++ .../SegmentNotWritableException.java | 28 + .../appenderator/SegmentsAndMetadata.java | 87 ++ .../druid/segment/realtime/plumber/Sink.java | 54 +- .../appenderator/AppenderatorTest.java | 453 +++++++ .../appenderator/AppenderatorTester.java | 274 ++++ .../realtime/appenderator/CommittedTest.java | 109 ++ 11 files changed, 2579 insertions(+), 17 deletions(-) create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/SegmentNotWritableException.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/SegmentsAndMetadata.java create mode 100644 server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java create mode 100644 server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java create mode 100644 server/src/test/java/io/druid/segment/realtime/appenderator/CommittedTest.java diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 4be0ccf4d918..6c4a956ca4ba 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -40,13 +40,17 @@ public class RealtimeTuningConfig implements TuningConfig private static final int defaultMaxRowsInMemory = 500000; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); private static final Period defaultWindowPeriod = new Period("PT10M"); - private static final File defaultBasePersistDirectory = Files.createTempDir(); private static final VersioningPolicy defaultVersioningPolicy = new IntervalStartVersioningPolicy(); private static final RejectionPolicyFactory defaultRejectionPolicyFactory = new ServerTimeRejectionPolicyFactory(); private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = new NoneShardSpec(); private static final IndexSpec defaultIndexSpec = new IndexSpec(); + private static File defaultBasePersistDirectory() + { + return Files.createTempDir(); + } + // Might make sense for this to be a builder public static RealtimeTuningConfig makeDefaultTuningConfig() { @@ -54,7 +58,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig() defaultMaxRowsInMemory, defaultIntermediatePersistPeriod, defaultWindowPeriod, - defaultBasePersistDirectory, + defaultBasePersistDirectory(), defaultVersioningPolicy, defaultRejectionPolicyFactory, defaultMaxPendingPersists, @@ -91,7 +95,7 @@ public RealtimeTuningConfig( ? defaultIntermediatePersistPeriod : intermediatePersistPeriod; this.windowPeriod = windowPeriod == null ? defaultWindowPeriod : windowPeriod; - this.basePersistDirectory = basePersistDirectory == null ? defaultBasePersistDirectory : basePersistDirectory; + this.basePersistDirectory = basePersistDirectory == null ? defaultBasePersistDirectory() : basePersistDirectory; this.versioningPolicy = versioningPolicy == null ? defaultVersioningPolicy : versioningPolicy; this.rejectionPolicyFactory = rejectionPolicyFactory == null ? defaultRejectionPolicyFactory @@ -184,4 +188,19 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) indexSpec ); } + + public RealtimeTuningConfig withShardSpec(ShardSpec newShardSpec) + { + return new RealtimeTuningConfig( + maxRowsInMemory, + intermediatePersistPeriod, + windowPeriod, + basePersistDirectory, + versioningPolicy, + rejectionPolicyFactory, + maxPendingPersists, + newShardSpec, + indexSpec + ); + } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java new file mode 100644 index 000000000000..e748f2b0b206 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -0,0 +1,171 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.google.common.base.Supplier; +import com.google.common.util.concurrent.ListenableFuture; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.query.QuerySegmentWalker; +import io.druid.segment.incremental.IndexSizeExceededException; + +import java.io.Closeable; +import java.util.List; + +/** + * Like a Plumber, but different. + *

+ * An Appenderator manages indexed data. It has some in-memory data and some persisted-on-disk data. It can serve + * queries on both of those. It can also push data to deep storage. But, it cannot publish segments to the metadata + * store; you have to do that yourself! + *

+ * Any time you call one of the methods that adds, persists, or pushes data, you must provide a Committer, or a + * Supplier of one, that represents all data you have given to the Appenderator so far. The Committer will be used when + * that data has been persisted to disk. + */ +public interface Appenderator extends QuerySegmentWalker, Closeable +{ + /** + * Return the name of the dataSource associated with this Appenderator. + */ + String getDataSource(); + + /** + * Perform any initial setup. Should be called before using any other methods. + * + * @return currently persisted commit metadata + */ + Object startJob(); + + /** + * Add a row. Must not be called concurrently from multiple threads. + *

+ * If no pending segment exists for the provided identifier, a new one will be created. + *

+ * This method may trigger a {@link #persistAll(Committer)} using the supplied Committer. If it does this, the + * Committer is guaranteed to be *created* synchronously with the call to add, but will actually be used + * asynchronously. + *

+ * The add, clear, persistAll, pushAll, and push methods should all be called from the same thread to ensure that + * commit metadata matches data that has actually been persisted to disk. + * + * @param identifier the segment into which this row should be added + * @param row the row to add + * @param committerSupplier supplier of a committer associated with all data that has been added, including this row + * + * @return positive number indicating how many summarized rows exist in this segment so far + * + * @throws IndexSizeExceededException if this row cannot be added because it is too large + * @throws SegmentNotWritableException if the requested segment is known, but has been closed + */ + int add(SegmentIdentifier identifier, InputRow row, Supplier committerSupplier) + throws IndexSizeExceededException, SegmentNotWritableException; + + /** + * Returns a list of all currently pending segments. + */ + List getSegments(); + + /** + * Returns the number of rows in a particular pending segment. + * + * @param identifier segment to examine + * + * @return row count + * + * @throws IllegalStateException if the segment is unknown + */ + int getRowCount(final SegmentIdentifier identifier); + + /** + * Drop all in-memory and on-disk data, and forget any previously-remembered commit metadata. This could be useful if, + * for some reason, rows have been added that we do not actually want to hand off. Blocks until all data has been + * cleared. This may take some time, since all pending persists must finish first. + */ + void clear() throws InterruptedException; + + /** + * Drop all data associated with a particular pending segment. Unlike {@link #clear()}), any on-disk commit + * metadata will remain unchanged. If there is no pending segment with this identifier, then this method will + * do nothing. + *

+ * You should not write to the dropped segment after calling "drop". If you need to drop all your data and + * re-write it, consider {@link #clear()} instead. + * + * @param identifier the pending segment to drop + * + * @return future that resolves when data is dropped + */ + ListenableFuture drop(SegmentIdentifier identifier); + + /** + * Persist any in-memory indexed data to durable storage. This may be only somewhat durable, e.g. the + * machine's local disk. The Committer will be made synchronously will the call to persistAll, but will actually + * be used asynchronously. Any metadata returned by the committer will be associated with the data persisted to + * disk. + *

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

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

+ * The add, clear, persistAll, pushAll, and push methods should all be called from the same thread to ensure that + * commit metadata matches data that has actually been persisted to disk. + * + * @param committer a committer associated with all data that has been added so far + * + * @return future that resolves when all segments have been pushed. The segment list will be the list of segments + * that have been pushed and the commit metadata from the Committer. + */ + ListenableFuture pushAll(Committer committer); + + /** + * Merge and push particular segments to deep storage. This will trigger an implicit {@link #persistAll(Committer)} + * using the provided Committer. + *

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

+ * The add, clear, persistAll, pushAll, and push methods should all be called from the same thread to ensure that + * commit metadata matches data that has actually been persisted to disk. + * + * @param identifiers list of segments to push + * @param committer a committer associated with all data that has been added so far + * + * @return future that resolves when all segments have been pushed. The segment list will be the list of segments + * that have been pushed and the commit metadata from the Committer. + */ + ListenableFuture push(List identifiers, Committer committer); + + /** + * Stop any currently-running processing and clean up after ourselves. This will not remove any on-disk persisted + * data, but it will drop any data that has not yet been persisted. + */ + void close(); +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java new file mode 100644 index 000000000000..ca25c67341eb --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -0,0 +1,1158 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.Pair; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.common.guava.ThreadRenamingCallable; +import io.druid.concurrent.Execs; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.query.BySegmentQueryRunner; +import io.druid.query.MetricsEmittingQueryRunner; +import io.druid.query.NoopQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.query.QueryToolChest; +import io.druid.query.ReferenceCountingSegmentQueryRunner; +import io.druid.query.ReportTimelineMissingSegmentQueryRunner; +import io.druid.query.SegmentDescriptor; +import io.druid.query.spec.SpecificSegmentQueryRunner; +import io.druid.query.spec.SpecificSegmentSpec; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMerger; +import io.druid.segment.IndexSpec; +import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IndexSizeExceededException; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.FireHydrant; +import io.druid.segment.realtime.plumber.Sink; +import io.druid.server.coordination.DataSegmentAnnouncer; +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 io.druid.timeline.partition.SingleElementPartitionChunk; +import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FilenameFilter; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.StandardOpenOption; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +/** + */ +public class AppenderatorImpl implements Appenderator +{ + private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class); + private static final int WARN_DELAY = 1000; + private static final String IDENTIFIER_FILE_NAME = "identifier.json"; + + private final DataSchema schema; + private final RealtimeTuningConfig config; + private final FireDepartmentMetrics metrics; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final QueryRunnerFactoryConglomerate conglomerate; + private final DataSegmentAnnouncer segmentAnnouncer; + private final ServiceEmitter emitter; + private final ExecutorService queryExecutorService; + private final IndexIO indexIO; + private final IndexMerger indexMerger; + private final Map sinks = Maps.newConcurrentMap(); + private final Set droppingSinks = Sets.newConcurrentHashSet(); + private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + + private volatile ListeningExecutorService persistExecutor = null; + private volatile ListeningExecutorService mergeExecutor = null; + private volatile long nextFlush; + private volatile FileLock basePersistDirLock = null; + private volatile FileChannel basePersistDirLockChannel = null; + + public AppenderatorImpl( + DataSchema schema, + RealtimeTuningConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + IndexIO indexIO, + IndexMerger indexMerger + ) + { + this.schema = Preconditions.checkNotNull(schema, "schema"); + this.config = Preconditions.checkNotNull(config, "config"); + this.metrics = Preconditions.checkNotNull(metrics, "metrics"); + this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); + this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); + this.conglomerate = conglomerate; + this.segmentAnnouncer = segmentAnnouncer; + this.emitter = emitter; + this.queryExecutorService = queryExecutorService; + this.indexIO = indexIO; + this.indexMerger = indexMerger; + + // If we're not querying (no conglomerate) then it's ok for the other query stuff to be null. + // But otherwise, we need them all. + if (conglomerate != null) { + Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); + Preconditions.checkNotNull(emitter, "emitter"); + Preconditions.checkNotNull(queryExecutorService, "queryExecutorService"); + } + + log.info("Creating appenderator for dataSource[%s]", schema.getDataSource()); + } + + public DataSchema getSchema() + { + return schema; + } + + public RealtimeTuningConfig getConfig() + { + return config; + } + + @Override + public String getDataSource() + { + return schema.getDataSource(); + } + + @Override + public Object startJob() + { + config.getBasePersistDirectory().mkdirs(); + lockBasePersistDirectory(); + final Object retVal = bootstrapSinksFromDisk(); + initializeExecutors(); + resetNextFlush(); + return retVal; + } + + @Override + public int add( + final SegmentIdentifier identifier, + final InputRow row, + final Supplier committerSupplier + ) throws IndexSizeExceededException, SegmentNotWritableException + { + if (!identifier.getDataSource().equals(schema.getDataSource())) { + throw new IAE( + "Expected dataSource[%s] but was asked to insert row for dataSource[%s]?!", + schema.getDataSource(), + identifier.getDataSource() + ); + } + + final Sink sink = getSink(identifier); + int numRows; + + try { + numRows = sink.add(row); + } + catch (IndexSizeExceededException e) { + // Try one more time after swapping, then throw the exception out if it happens again. + persistAll(committerSupplier.get()); + numRows = sink.add(row); + } + + if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush) { + persistAll(committerSupplier.get()); + } + + if (numRows < 0) { + throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier); + } else { + return numRows; + } + } + + @Override + public List getSegments() + { + return ImmutableList.copyOf(sinks.keySet()); + } + + @Override + public int getRowCount(final SegmentIdentifier identifier) + { + final Sink sink = sinks.get(identifier); + + if (sink == null) { + throw new ISE("No such sink: %s", identifier); + } + + int retVal = 0; + for (FireHydrant hydrant : sink) { + final IncrementalIndex index = hydrant.getIndex(); + if (index != null) { + retVal += index.size(); + } else { + retVal += hydrant.getSegment().asQueryableIndex().getNumRows(); + } + } + + return retVal; + } + + private Sink getSink(final SegmentIdentifier identifier) + { + Sink retVal = sinks.get(identifier); + + if (retVal == null) { + retVal = new Sink( + identifier.getInterval(), + schema, + config.withShardSpec(identifier.getShardSpec()), + identifier.getVersion() + ); + + try { + segmentAnnouncer.announceSegment(retVal.getSegment()); + } + catch (IOException e) { + log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) + .addData("interval", retVal.getInterval()) + .emit(); + } + + sinks.put(identifier, retVal); + sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), identifier.getShardSpec().createChunk(retVal)); + } + + return retVal; + } + + @Override + public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) + { + if (conglomerate == null) { + throw new IllegalStateException("Don't query me, bro."); + } + + final Iterable specs = FunctionalIterable + .create(intervals) + .transformCat( + new Function>>() + { + @Override + public Iterable> apply(final Interval interval) + { + return sinkTimeline.lookup(interval); + } + } + ) + .transformCat( + new Function, Iterable>() + { + @Override + public Iterable apply(final TimelineObjectHolder holder) + { + return FunctionalIterable + .create(holder.getObject()) + .transform( + new Function, SegmentDescriptor>() + { + @Override + public SegmentDescriptor apply(final PartitionChunk chunk) + { + return new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + chunk.getChunkNumber() + ); + } + } + ); + } + } + ); + + return getQueryRunnerForSegments(query, specs); + } + + @Override + public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) + { + if (conglomerate == null) { + throw new IllegalStateException("Don't query me, bro."); + } + + // We only handle one dataSource. Make sure it's in the list of names, then ignore from here on out. + if (!query.getDataSource().getNames().contains(getDataSource())) { + log.makeAlert("Received query for unknown dataSource") + .addData("dataSource", query.getDataSource()) + .emit(); + return new NoopQueryRunner<>(); + } + + final QueryRunnerFactory> factory = conglomerate.findFactory(query); + if (factory == null) { + log.makeAlert("Unknown query type, [%s]", query.getClass()) + .addData("dataSource", query.getDataSource()) + .emit(); + return new NoopQueryRunner<>(); + } + + final QueryToolChest> toolchest = factory.getToolchest(); + final Function, ServiceMetricEvent.Builder> builderFn = + new Function, ServiceMetricEvent.Builder>() + { + + @Override + public ServiceMetricEvent.Builder apply(@Nullable Query input) + { + return toolchest.makeMetricBuilder(query); + } + }; + + return toolchest.mergeResults( + factory.mergeRunners( + queryExecutorService, + FunctionalIterable + .create(specs) + .transform( + new Function>() + { + @Override + public QueryRunner apply(final SegmentDescriptor spec) + { + final PartitionHolder holder = sinkTimeline.findEntry( + spec.getInterval(), + spec.getVersion() + ); + if (holder == null) { + return new ReportTimelineMissingSegmentQueryRunner<>(spec); + } + + final PartitionChunk chunk = holder.getChunk(spec.getPartitionNumber()); + if (chunk == null) { + return new ReportTimelineMissingSegmentQueryRunner<>(spec); + } + + final Sink theSink = chunk.getObject(); + + return new SpecificSegmentQueryRunner<>( + new MetricsEmittingQueryRunner<>( + emitter, + builderFn, + new BySegmentQueryRunner( + theSink.getSegment().getIdentifier(), + spec.getInterval().getStart(), + factory.mergeRunners( + MoreExecutors.sameThreadExecutor(), + Iterables.transform( + theSink, + new Function>() + { + @Override + public QueryRunner apply(final FireHydrant hydrant) + { + // TODO: Make sure this still works when hydrants are actually closed + return new ReferenceCountingSegmentQueryRunner<>( + factory, + hydrant.getSegment() + ); + } + } + ) + ) + ) + ).withWaitMeasuredFromNow(), + new SpecificSegmentSpec(spec) + ); + } + } + ) + ) + ); + } + + @Override + public void clear() throws InterruptedException + { + // Drop commit metadata, then abandon all segments. + + try { + final ListenableFuture uncommitFuture = persistExecutor.submit( + new Callable() + { + @Override + public Object call() throws Exception + { + objectMapper.writeValue(computeCommitFile(), Committed.nil()); + return null; + } + } + ); + + // Await uncommit. + uncommitFuture.get(); + + // Drop everything. + final List> futures = Lists.newArrayList(); + for (Map.Entry entry : sinks.entrySet()) { + futures.add(abandonSegment(entry.getKey(), entry.getValue(), true)); + } + + // Await dropping. + Futures.allAsList(futures).get(); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + + @Override + public ListenableFuture drop(final SegmentIdentifier identifier) + { + final Sink sink = sinks.get(identifier); + if (sink != null) { + return abandonSegment(identifier, sink, true); + } else { + return Futures.immediateFuture(null); + } + } + + @Override + public ListenableFuture persistAll(final Committer committer) + { + // Submit persistAll task to the persistExecutor + + final Map commitHydrants = Maps.newHashMap(); + final List> indexesToPersist = Lists.newArrayList(); + final Set identifiers = sinks.keySet(); + for (SegmentIdentifier identifier : identifiers) { + final Sink sink = sinks.get(identifier); + final List hydrants = Lists.newArrayList(sink); + commitHydrants.put(identifier, hydrants.size()); + + final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size(); + + for (FireHydrant hydrant : hydrants.subList(0, limit)) { + if (!hydrant.hasSwapped()) { + log.info("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier); + indexesToPersist.add(Pair.of(hydrant, identifier)); + } + } + + if (sink.swappable()) { + indexesToPersist.add(Pair.of(sink.swap(), identifier)); + } + } + + log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); + + final String threadName = String.format("%s-incremental-persist", schema.getDataSource()); + final Stopwatch runExecStopwatch = Stopwatch.createStarted(); + final Stopwatch persistStopwatch = Stopwatch.createStarted(); + final ListenableFuture future = persistExecutor.submit( + new ThreadRenamingCallable(threadName) + { + @Override + public Object doCall() + { + try { + for (Pair pair : indexesToPersist) { + metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs)); + } + + log.info( + "Committing metadata[%s] for sinks[%s].", committer.getMetadata(), Joiner.on(", ").join( + Iterables.transform( + commitHydrants.entrySet(), + new Function, String>() + { + @Override + public String apply(Map.Entry entry) + { + return String.format("%s:%d", entry.getKey().getIdentifierAsString(), entry.getValue()); + } + } + ) + ) + ); + + committer.run(); + objectMapper.writeValue(computeCommitFile(), Committed.create(commitHydrants, committer.getMetadata())); + + return committer.getMetadata(); + } + catch (Exception e) { + metrics.incrementFailedPersists(); + throw Throwables.propagate(e); + } + finally { + metrics.incrementNumPersists(); + metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); + persistStopwatch.stop(); + } + } + } + ); + + final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); + metrics.incrementPersistBackPressureMillis(startDelay); + if (startDelay > WARN_DELAY) { + log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); + } + runExecStopwatch.stop(); + resetNextFlush(); + + return future; + } + + @Override + public ListenableFuture pushAll(Committer committer) + { + return push(Lists.newArrayList(sinks.keySet()), committer); + } + + @Override + public ListenableFuture push( + final List identifiers, + final Committer committer + ) + { + final Map theSinks = Maps.newHashMap(); + for (final SegmentIdentifier identifier : identifiers) { + final Sink sink = sinks.get(identifier); + if (sink == null) { + throw new NullPointerException("No sink for identifier: " + identifier); + } + theSinks.put(identifier, sink); + sink.finishWriting(); + } + + return Futures.transform( + persistAll(committer), + new Function() + { + @Override + public SegmentsAndMetadata apply(Object commitMetadata) + { + final List dataSegments = Lists.newArrayList(); + + for (Map.Entry entry : theSinks.entrySet()) { + if (droppingSinks.contains(entry.getKey())) { + log.info("Skipping push of currently-dropping sink[%s]", entry.getKey()); + continue; + } + + final DataSegment dataSegment = mergeAndPush(entry.getKey(), entry.getValue()); + if (dataSegment != null) { + dataSegments.add(dataSegment); + } else { + log.warn("mergeAndPush[%s] returned null, skipping.", entry.getKey()); + } + } + + return new SegmentsAndMetadata(dataSegments, commitMetadata); + } + }, + mergeExecutor + ); + } + + /** + * Insert a barrier into the merge-and-push queue. When this future resolves, all pending pushes will have finished. + * This is useful if we're going to do something that would otherwise potentially break currently in-progress + * pushes. + */ + private ListenableFuture mergeBarrier() + { + return mergeExecutor.submit( + new Runnable() + { + @Override + public void run() + { + // Do nothing + } + } + ); + } + + /** + * Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only + * be run in the single-threaded mergeExecutor. + * + * @param identifier sink identifier + * @param sink sink to push + * + * @return segment descriptor, or null if the sink is no longer valid + */ + + private DataSegment mergeAndPush(final SegmentIdentifier identifier, final Sink sink) + { + // Bail out if this sink is null or otherwise not what we expect. + if (sinks.get(identifier) != sink) { + log.warn("Sink for segment[%s] no longer valid, bailing out of mergeAndPush.", identifier); + return null; + } + + // Use a descriptor file to indicate that pushing has completed. + final File persistDir = computePersistDir(identifier); + final File mergedTarget = new File(persistDir, "merged"); + final File descriptorFile = computeDescriptorFile(identifier); + + // Sanity checks + for (FireHydrant hydrant : sink) { + if (sink.isWritable()) { + throw new ISE("WTF?! Expected sink to be no longer writable before mergeAndPush. Segment[%s].", identifier); + } + + synchronized (hydrant) { + if (!hydrant.hasSwapped()) { + throw new ISE("WTF?! Expected sink to be fully persisted before mergeAndPush. Segment[%s].", identifier); + } + } + } + + try { + if (descriptorFile.exists()) { + // Already pushed. + return objectMapper.readValue(descriptorFile, DataSegment.class); + } + + log.info("Pushing merged index for segment[%s].", identifier); + + removeDirectory(mergedTarget); + + if (mergedTarget.exists()) { + throw new ISE("Merged target[%s] exists after removing?!", mergedTarget); + } + + List indexes = Lists.newArrayList(); + for (FireHydrant fireHydrant : sink) { + Segment segment = fireHydrant.getSegment(); + final QueryableIndex queryableIndex = segment.asQueryableIndex(); + log.info("Adding hydrant[%s]", fireHydrant); + indexes.add(queryableIndex); + } + + final File mergedFile; + mergedFile = indexMerger.mergeQueryableIndex( + indexes, + schema.getAggregators(), + mergedTarget, + config.getIndexSpec() + ); + + QueryableIndex index = indexIO.loadIndex(mergedFile); + + DataSegment segment = dataSegmentPusher.push( + mergedFile, + sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) + ); + + objectMapper.writeValue(descriptorFile, segment); + + log.info("Pushed merged index for segment[%s], descriptor is: %s", identifier, segment); + + return segment; + } + catch (Exception e) { + metrics.incrementFailedHandoffs(); + log.warn(e, "Failed to push merged index for segment[%s].", identifier); + throw Throwables.propagate(e); + } + } + + @Override + public void close() + { + log.info("Shutting down..."); + + final List> futures = Lists.newArrayList(); + for (Map.Entry entry : sinks.entrySet()) { + futures.add(abandonSegment(entry.getKey(), entry.getValue(), false)); + } + + try { + Futures.allAsList(futures).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.warn(e, "Interrupted during close()"); + } + catch (ExecutionException e) { + log.warn(e, "Unable to abandon existing segments during close()"); + } + + try { + shutdownExecutors(); + Preconditions.checkState(persistExecutor.awaitTermination(365, TimeUnit.DAYS), "persistExecutor not terminated"); + Preconditions.checkState(mergeExecutor.awaitTermination(365, TimeUnit.DAYS), "mergeExecutor not terminated"); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ISE("Failed to shutdown executors during close()"); + } + + // Only unlock if executors actually shut down. + unlockBasePersistDirectory(); + } + + private void lockBasePersistDirectory() + { + if (basePersistDirLock == null) { + try { + basePersistDirLockChannel = FileChannel.open( + computeLockFile().toPath(), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE + ); + + basePersistDirLock = basePersistDirLockChannel.tryLock(); + if (basePersistDirLock == null) { + throw new ISE("Cannot acquire lock on basePersistDir: %s", computeLockFile()); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + + private void unlockBasePersistDirectory() + { + try { + if (basePersistDirLock != null) { + basePersistDirLock.release(); + basePersistDirLockChannel.close(); + basePersistDirLock = null; + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private void initializeExecutors() + { + final int maxPendingPersists = config.getMaxPendingPersists(); + + if (persistExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + persistExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "appenderator_persist_%d", maxPendingPersists + ) + ); + } + if (mergeExecutor == null) { + // use a blocking single threaded executor to throttle the firehose when write to disk is slow + mergeExecutor = MoreExecutors.listeningDecorator( + Execs.newBlockingSingleThreaded( + "appenderator_merge_%d", 1 + ) + ); + } + } + + private void shutdownExecutors() + { + persistExecutor.shutdownNow(); + mergeExecutor.shutdownNow(); + } + + private void resetNextFlush() + { + nextFlush = new DateTime().plus(config.getIntermediatePersistPeriod()).getMillis(); + } + + /** + * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer. + * + * @return persisted commit metadata + */ + private Object bootstrapSinksFromDisk() + { + // TODO: Convert old sink directories to new ones (assuming shardSpec is known upfront)? + + Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!"); + + final File baseDir = config.getBasePersistDirectory(); + if (!baseDir.exists()) { + return null; + } + + final File[] files = baseDir.listFiles(); + if (files == null) { + return null; + } + + final File commitFile = computeCommitFile(); + final Committed committed; + try { + if (commitFile.exists()) { + committed = objectMapper.readValue(commitFile, Committed.class); + } else { + committed = Committed.nil(); + } + } + catch (Exception e) { + throw new ISE(e, "Failed to read commitFile: %s", commitFile); + } + + log.info("Loading sinks: %s", committed.getHydrants().keySet()); + + for (File sinkDir : files) { + final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME); + if (!identifierFile.isFile()) { + // No identifier in this sinkDir; it must not actually be a sink directory. Skip it. + continue; + } + + try { + final SegmentIdentifier identifier = objectMapper.readValue( + new File(sinkDir, "identifier.json"), + SegmentIdentifier.class + ); + + final int committedHydrants = committed.getCommittedHydrants(identifier.getIdentifierAsString()); + + if (committedHydrants <= 0) { + log.info("Removing uncommitted sink at [%s]", sinkDir); + FileUtils.deleteDirectory(sinkDir); + continue; + } + + // To avoid reading and listing of "merged" dir and other special files + final File[] sinkFiles = sinkDir.listFiles( + new FilenameFilter() + { + @Override + public boolean accept(File dir, String fileName) + { + return !(Ints.tryParse(fileName) == null); + } + } + ); + + Arrays.sort( + sinkFiles, + new Comparator() + { + @Override + public int compare(File o1, File o2) + { + return Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())); + } + } + ); + + List hydrants = Lists.newArrayList(); + for (File hydrantDir : sinkFiles) { + final int hydrantNumber = Integer.parseInt(hydrantDir.getName()); + + if (hydrantNumber >= committedHydrants) { + log.info("Removing uncommitted segment at [%s]", hydrantDir); + FileUtils.deleteDirectory(hydrantDir); + } else { + log.info("Loading previously persisted segment at [%s]", hydrantDir); + if (hydrantNumber != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); + } + + hydrants.add( + new FireHydrant( + new QueryableIndexSegment( + identifier.getIdentifierAsString(), + indexIO.loadIndex(hydrantDir) + ), + hydrantNumber + ) + ); + } + } + + // Make sure we loaded enough hydrants. + if (committedHydrants != hydrants.size()) { + throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir); + } + + Sink currSink = new Sink(identifier.getInterval(), schema, config, identifier.getVersion(), hydrants); + sinks.put(identifier, currSink); + sinkTimeline.add( + currSink.getInterval(), + currSink.getVersion(), + new SingleElementPartitionChunk<>(currSink) + ); + + segmentAnnouncer.announceSegment(currSink.getSegment()); + } + catch (IOException e) { + log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource()) + .addData("sinkDir", sinkDir) + .emit(); + } + } + + // Make sure we loaded all committed sinks. + final Set loadedSinks = Sets.newHashSet( + Iterables.transform( + sinks.keySet(), + new Function() + { + @Override + public String apply(SegmentIdentifier input) + { + return input.getIdentifierAsString(); + } + } + ) + ); + final Set missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks); + if (!missingSinks.isEmpty()) { + throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks)); + } + + return committed.getMetadata(); + } + + private ListenableFuture abandonSegment( + final SegmentIdentifier identifier, + final Sink sink, + final boolean removeOnDiskData + ) + { + // Mark this identifier as dropping, so no future merge tasks will pick it up. + droppingSinks.add(identifier); + + // Wait for any outstanding merges to finish, then abandon the segment inside the persist thread. + return Futures.transform( + mergeBarrier(), + new Function() + { + @Nullable + @Override + public Object apply(@Nullable Object input) + { + if (sinks.get(identifier) != sink) { + // Only abandon sink if it is the same one originally requested to be abandoned. + log.warn("Sink for segment[%s] no longer valid, not abandoning."); + return null; + } + + if (removeOnDiskData) { + // Remove this segment from the committed list. This must be done from the persist thread. + log.info("Removing commit metadata for segment[%s].", identifier); + try { + final File commitFile = computeCommitFile(); + if (commitFile.exists()) { + final Committed oldCommitted = objectMapper.readValue(commitFile, Committed.class); + objectMapper.writeValue(commitFile, oldCommitted.without(identifier.getIdentifierAsString())); + } + } + catch (Exception e) { + log.makeAlert(e, "Failed to update committed segments[%s]", schema.getDataSource()) + .addData("identifier", identifier.getIdentifierAsString()) + .emit(); + throw Throwables.propagate(e); + } + } + + // Unannounce the segment. + try { + segmentAnnouncer.unannounceSegment(sink.getSegment()); + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource()) + .addData("identifier", identifier.getIdentifierAsString()) + .emit(); + } + + log.info("Removing sink for segment[%s].", identifier); + sinks.remove(identifier); + droppingSinks.remove(identifier); + sinkTimeline.remove( + sink.getInterval(), + sink.getVersion(), + new SingleElementPartitionChunk<>(sink) + ); + + if (removeOnDiskData) { + removeDirectory(computePersistDir(identifier)); + } + + return null; + } + }, + persistExecutor + ); + } + + private File computeCommitFile() + { + return new File(config.getBasePersistDirectory(), "commit.json"); + } + + private File computeLockFile() + { + return new File(config.getBasePersistDirectory(), ".lock"); + } + + private File computePersistDir(SegmentIdentifier identifier) + { + return new File(config.getBasePersistDirectory(), identifier.getIdentifierAsString()); + } + + private File computeIdentifierFile(SegmentIdentifier identifier) + { + return new File(computePersistDir(identifier), IDENTIFIER_FILE_NAME); + } + + private File computeDescriptorFile(SegmentIdentifier identifier) + { + return new File(computePersistDir(identifier), "descriptor.json"); + } + + private File createPersistDirIfNeeded(SegmentIdentifier identifier) throws IOException + { + final File persistDir = computePersistDir(identifier); + if (!persistDir.mkdir() && !persistDir.exists()) { + throw new IOException(String.format("Could not create directory: %s", persistDir)); + } + + objectMapper.writeValue(computeIdentifierFile(identifier), identifier); + + return persistDir; + } + + /** + * Persists the given hydrant and returns the number of rows persisted. Must only be called in the single-threaded + * persistExecutor. + * + * @param indexToPersist hydrant to persist + * @param identifier the segment this hydrant is going to be part of + * + * @return the number of rows persisted + */ + private int persistHydrant(FireHydrant indexToPersist, SegmentIdentifier identifier) + { + synchronized (indexToPersist) { + if (indexToPersist.hasSwapped()) { + log.info( + "Segment[%s], Hydrant[%s] already swapped. Ignoring request to persist.", + identifier, indexToPersist + ); + return 0; + } + + log.info("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist); + + try { + int numRows = indexToPersist.getIndex().size(); + + final File persistedFile; + final File persistDir = createPersistDirIfNeeded(identifier); + final IndexSpec indexSpec = config.getIndexSpec(); + persistedFile = indexMerger.persist( + indexToPersist.getIndex(), + new File(persistDir, String.valueOf(indexToPersist.getCount())), + null, + indexSpec + ); + + indexToPersist.swapSegment( + new QueryableIndexSegment( + indexToPersist.getSegment().getIdentifier(), + indexIO.loadIndex(persistedFile) + ) + ); + return numRows; + } + catch (IOException e) { + log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource()) + .addData("segment", identifier.getIdentifierAsString()) + .addData("count", indexToPersist.getCount()) + .emit(); + + throw Throwables.propagate(e); + } + } + } + + private void removeDirectory(final File target) + { + if (target.exists()) { + try { + log.info("Deleting Index File[%s]", target); + FileUtils.deleteDirectory(target); + } + catch (Exception e) { + log.makeAlert(e, "Failed to remove directory[%s]", schema.getDataSource()) + .addData("file", target) + .emit(); + } + } + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java new file mode 100644 index 000000000000..8d2061487146 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java @@ -0,0 +1,118 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMaker; +import io.druid.segment.IndexMerger; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.timeline.DataSegment; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; + +public class Appenderators +{ + public static Appenderator realtime( + DataSchema schema, + RealtimeTuningConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService + ) + { + return new AppenderatorImpl( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + conglomerate, + segmentAnnouncer, + emitter, + queryExecutorService, + indexIO, + indexMerger + ); + } + + public static Appenderator offline( + DataSchema schema, + RealtimeTuningConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger + ) + { + return new AppenderatorImpl( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + null, + new DataSegmentAnnouncer() + { + @Override + public void announceSegment(DataSegment segment) throws IOException + { + // Do nothing + } + + @Override + public void unannounceSegment(DataSegment segment) throws IOException + { + // Do nothing + } + + @Override + public void announceSegments(Iterable segments) throws IOException + { + // Do nothing + } + + @Override + public void unannounceSegments(Iterable segments) throws IOException + { + // Do nothing + } + }, + null, + null, + indexIO, + indexMerger + ); + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java new file mode 100644 index 000000000000..14f13cdbe6a3 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Committed.java @@ -0,0 +1,119 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +import java.util.Map; +import java.util.Objects; + +public class Committed +{ + private static final Committed NIL = new Committed(ImmutableMap.of(), null); + + // Map of segment identifierAsString -> number of committed hydrants + private final ImmutableMap hydrants; + private final Object metadata; + + @JsonCreator + public Committed( + @JsonProperty("hydrants") Map hydrants, + @JsonProperty("metadata") Object metadata + ) + { + this.hydrants = ImmutableMap.copyOf(hydrants); + this.metadata = metadata; + } + + public static Committed create( + Map hydrants0, + Object metadata + ) + { + final ImmutableMap.Builder hydrants = ImmutableMap.builder(); + for (Map.Entry entry : hydrants0.entrySet()) { + hydrants.put(entry.getKey().getIdentifierAsString(), entry.getValue()); + } + return new Committed(hydrants.build(), metadata); + } + + @JsonProperty + public ImmutableMap getHydrants() + { + return hydrants; + } + + @JsonProperty + public Object getMetadata() + { + return metadata; + } + + public int getCommittedHydrants(final String identifierAsString) + { + final Integer committedHydrant = hydrants.get(identifierAsString); + return committedHydrant == null ? 0 : committedHydrant; + } + + public Committed without(final String identifierAsString) + { + final Map newHydrants = Maps.newHashMap(); + newHydrants.putAll(hydrants); + newHydrants.remove(identifierAsString); + return new Committed(newHydrants, metadata); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Committed committed = (Committed) o; + return Objects.equals(hydrants, committed.hydrants) && + Objects.equals(metadata, committed.metadata); + } + + @Override + public int hashCode() + { + return Objects.hash(hydrants, metadata); + } + + @Override + public String toString() + { + return "Committed{" + + "hydrants=" + hydrants + + ", metadata=" + metadata + + '}'; + } + + public static Committed nil() + { + return NIL; + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentNotWritableException.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentNotWritableException.java new file mode 100644 index 000000000000..354866a5cd4f --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentNotWritableException.java @@ -0,0 +1,28 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +public class SegmentNotWritableException extends Exception +{ + public SegmentNotWritableException(String message, Object... messageArgs) + { + super(String.format(message, messageArgs)); + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentsAndMetadata.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentsAndMetadata.java new file mode 100644 index 000000000000..f5d7ae1e1902 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentsAndMetadata.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.segment.realtime.appenderator; + +import com.google.common.collect.ImmutableList; +import io.druid.timeline.DataSegment; + +import java.util.List; +import java.util.Objects; + +public class SegmentsAndMetadata +{ + private static final SegmentsAndMetadata NIL = new SegmentsAndMetadata(ImmutableList.of(), null); + + private final Object commitMetadata; + private final ImmutableList segments; + + public SegmentsAndMetadata( + List segments, + Object commitMetadata + ) + { + this.segments = ImmutableList.copyOf(segments); + this.commitMetadata = commitMetadata; + } + + public Object getCommitMetadata() + { + return commitMetadata; + } + + public List getSegments() + { + return segments; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentsAndMetadata that = (SegmentsAndMetadata) o; + return Objects.equals(commitMetadata, that.commitMetadata) && + Objects.equals(segments, that.segments); + } + + @Override + public int hashCode() + { + return Objects.hash(commitMetadata, segments); + } + + @Override + public String toString() + { + return "SegmentsAndMetadata{" + + "commitMetadata=" + commitMetadata + + ", segments=" + segments + + '}'; + } + + public static SegmentsAndMetadata nil() + { + return NIL; + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java index ca3951e2c936..104595ba6b36 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java @@ -44,8 +44,6 @@ import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; -/** - */ public class Sink implements Iterable { @@ -56,6 +54,7 @@ public class Sink implements Iterable private final String version; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList(); private volatile FireHydrant currHydrant; + private volatile boolean writable = true; public Sink( Interval interval, @@ -120,6 +119,10 @@ public int add(InputRow row) throws IndexSizeExceededException } synchronized (hydrantLock) { + if (!writable) { + return -1; + } + IncrementalIndex index = currHydrant.getIndex(); if (index == null) { return -1; // the hydrant was swapped without being replaced @@ -130,8 +133,8 @@ public int add(InputRow row) throws IndexSizeExceededException public boolean canAppendRow() { - synchronized (currHydrant) { - return currHydrant != null && currHydrant.getIndex().canAppendRow(); + synchronized (hydrantLock) { + return writable && currHydrant != null && currHydrant.getIndex().canAppendRow(); } } @@ -142,6 +145,11 @@ public boolean isEmpty() } } + public boolean isWritable() + { + return writable; + } + /** * If currHydrant is A, creates a new index B, sets currHydrant to B and returns A. * @@ -155,7 +163,19 @@ public FireHydrant swap() public boolean swappable() { synchronized (hydrantLock) { - return currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0; + return writable && currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0; + } + } + + public boolean finished() + { + return !writable; + } + + public void finishWriting() + { + synchronized (hydrantLock) { + writable = false; } } @@ -198,15 +218,21 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) final FireHydrant old; synchronized (hydrantLock) { - old = currHydrant; - int newCount = 0; - int numHydrants = hydrants.size(); - if (numHydrants > 0) { - FireHydrant lastHydrant = hydrants.get(numHydrants - 1); - newCount = lastHydrant.getCount() + 1; + if (writable) { + old = currHydrant; + int newCount = 0; + int numHydrants = hydrants.size(); + if (numHydrants > 0) { + FireHydrant lastHydrant = hydrants.get(numHydrants - 1); + newCount = lastHydrant.getCount() + 1; + } + currHydrant = new FireHydrant(newIndex, newCount, getSegment().getIdentifier()); + hydrants.add(currHydrant); + } else { + // Oops, someone called finishWriting while we were making this new index. + newIndex.close(); + throw new ISE("finishWriting() called during swap"); } - currHydrant = new FireHydrant(newIndex, newCount, getSegment().getIdentifier()); - hydrants.add(currHydrant); } return old; @@ -220,7 +246,7 @@ public Iterator iterator() new Predicate() { @Override - public boolean apply(@Nullable FireHydrant input) + public boolean apply(FireHydrant input) { final IncrementalIndex index = input.getIndex(); return index == null || index.size() != 0; diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java new file mode 100644 index 000000000000..3485c9102d41 --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -0,0 +1,453 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.metamx.common.guava.Sequences; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.Result; +import io.druid.query.SegmentDescriptor; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.spec.MultipleSpecificSegmentSpec; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.realtime.plumber.Committers; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + +public class AppenderatorTest +{ + private static final List IDENTIFIERS = ImmutableList.of( + SI("2000/2001", "A", 0), + SI("2000/2001", "A", 1), + SI("2001/2002", "A", 0) + ); + + @Test + public void testSimpleIngestion() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(2)) { + final Appenderator appenderator = tester.getAppenderator(); + boolean thrown; + + final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); + final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); + + // startJob + Assert.assertEquals(null, appenderator.startJob()); + + // getDataSource + Assert.assertEquals(AppenderatorTester.DATASOURCE, appenderator.getDataSource()); + + // add + commitMetadata.put("x", "1"); + Assert.assertEquals(1, appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier)); + + commitMetadata.put("x", "2"); + Assert.assertEquals(2, appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier)); + + commitMetadata.put("x", "3"); + Assert.assertEquals(1, appenderator.add(IDENTIFIERS.get(1), IR("2000", "qux", 4), committerSupplier)); + + // getSegments + Assert.assertEquals(IDENTIFIERS.subList(0, 2), sorted(appenderator.getSegments())); + + // getRowCount + Assert.assertEquals(2, appenderator.getRowCount(IDENTIFIERS.get(0))); + Assert.assertEquals(1, appenderator.getRowCount(IDENTIFIERS.get(1))); + thrown = false; + try { + appenderator.getRowCount(IDENTIFIERS.get(2)); + } + catch (IllegalStateException e) { + thrown = true; + } + Assert.assertTrue(thrown); + + // pushAll + final SegmentsAndMetadata segmentsAndMetadata = appenderator.pushAll(committerSupplier.get()).get(); + Assert.assertEquals(ImmutableMap.of("x", "3"), (Map) segmentsAndMetadata.getCommitMetadata()); + Assert.assertEquals( + IDENTIFIERS.subList(0, 2), + sorted( + Lists.transform( + segmentsAndMetadata.getSegments(), + new Function() + { + @Override + public SegmentIdentifier apply(DataSegment input) + { + return SegmentIdentifier.fromDataSegment(input); + } + } + ) + ) + ); + Assert.assertEquals(sorted(tester.getPushedSegments()), sorted(segmentsAndMetadata.getSegments())); + + // clear + appenderator.clear(); + Assert.assertTrue(appenderator.getSegments().isEmpty()); + } + } + + @Test + public void testRestoreFromDisk() throws Exception + { + final RealtimeTuningConfig tuningConfig; + try (final AppenderatorTester tester = new AppenderatorTester(2)) { + final Appenderator appenderator = tester.getAppenderator(); + tuningConfig = tester.getTuningConfig(); + + final AtomicInteger eventCount = new AtomicInteger(0); + final Supplier committerSupplier = new Supplier() + { + @Override + public Committer get() + { + final Object metadata = ImmutableMap.of("eventCount", eventCount.get()); + + return new Committer() + { + @Override + public Object getMetadata() + { + return metadata; + } + + @Override + public void run() + { + // Do nothing + } + }; + } + }; + + appenderator.startJob(); + eventCount.incrementAndGet(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), committerSupplier); + eventCount.incrementAndGet(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "bar", 2), committerSupplier); + eventCount.incrementAndGet(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "baz", 3), committerSupplier); + eventCount.incrementAndGet(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "qux", 4), committerSupplier); + eventCount.incrementAndGet(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "bob", 5), committerSupplier); + appenderator.close(); + + try (final AppenderatorTester tester2 = new AppenderatorTester(2, tuningConfig.getBasePersistDirectory())) { + final Appenderator appenderator2 = tester2.getAppenderator(); + Assert.assertEquals(ImmutableMap.of("eventCount", 4), appenderator2.startJob()); + Assert.assertEquals(ImmutableList.of(IDENTIFIERS.get(0)), appenderator2.getSegments()); + Assert.assertEquals(4, appenderator2.getRowCount(IDENTIFIERS.get(0))); + } + } + } + + @Test + public void testQueryByIntervals() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(2)) { + final Appenderator appenderator = tester.getAppenderator(); + + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 2), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(1), IR("2000", "foo", 4), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(2), IR("2001", "foo", 8), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(2), IR("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(2), IR("2001T02", "foo", 32), Suppliers.ofInstance(Committers.nil())); + + // Query1: 2000/2001 + final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() + .dataSource(AppenderatorTester.DATASOURCE) + .intervals(ImmutableList.of(new Interval("2000/2001"))) + .aggregators( + Arrays.asList( + new LongSumAggregatorFactory("count", "count"), + new LongSumAggregatorFactory("met", "met") + ) + ) + .granularity(QueryGranularity.DAY) + .build(); + + final List> results1 = Lists.newArrayList(); + Sequences.toList(query1.run(appenderator, ImmutableMap.of()), results1); + Assert.assertEquals( + "query1", + ImmutableList.of( + new Result<>( + new DateTime("2000"), + new TimeseriesResultValue(ImmutableMap.of("count", 3L, "met", 7L)) + ) + ), + results1 + ); + + // Query2: 2000/2002 + final TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder() + .dataSource(AppenderatorTester.DATASOURCE) + .intervals(ImmutableList.of(new Interval("2000/2002"))) + .aggregators( + Arrays.asList( + new LongSumAggregatorFactory("count", "count"), + new LongSumAggregatorFactory("met", "met") + ) + ) + .granularity(QueryGranularity.DAY) + .build(); + + final List> results2 = Lists.newArrayList(); + Sequences.toList(query2.run(appenderator, ImmutableMap.of()), results2); + Assert.assertEquals( + "query2", + ImmutableList.of( + new Result<>( + new DateTime("2000"), + new TimeseriesResultValue(ImmutableMap.of("count", 3L, "met", 7L)) + ), + new Result<>( + new DateTime("2001"), + new TimeseriesResultValue(ImmutableMap.of("count", 3L, "met", 56L)) + ) + ), + results2 + ); + + // Query3: 2000/2001T01 + final TimeseriesQuery query3 = Druids.newTimeseriesQueryBuilder() + .dataSource(AppenderatorTester.DATASOURCE) + .intervals(ImmutableList.of(new Interval("2000/2001T01"))) + .aggregators( + Arrays.asList( + new LongSumAggregatorFactory("count", "count"), + new LongSumAggregatorFactory("met", "met") + ) + ) + .granularity(QueryGranularity.DAY) + .build(); + + final List> results3 = Lists.newArrayList(); + Sequences.toList(query3.run(appenderator, ImmutableMap.of()), results3); + Assert.assertEquals( + ImmutableList.of( + new Result<>( + new DateTime("2000"), + new TimeseriesResultValue(ImmutableMap.of("count", 3L, "met", 7L)) + ), + new Result<>( + new DateTime("2001"), + new TimeseriesResultValue(ImmutableMap.of("count", 1L, "met", 8L)) + ) + ), + results3 + ); + } + } + + @Test + public void testQueryBySegments() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester(2)) { + final Appenderator appenderator = tester.getAppenderator(); + + appenderator.startJob(); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 1), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(0), IR("2000", "foo", 2), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(1), IR("2000", "foo", 4), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(2), IR("2001", "foo", 8), Suppliers.ofInstance(Committers.nil())); + appenderator.add(IDENTIFIERS.get(2), IR("2001T01", "foo", 16), Suppliers.ofInstance(Committers.nil())); + + // Query1: segment #2 + final TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() + .dataSource(AppenderatorTester.DATASOURCE) + .aggregators( + Arrays.asList( + new LongSumAggregatorFactory("count", "count"), + new LongSumAggregatorFactory("met", "met") + ) + ) + .granularity(QueryGranularity.DAY) + .intervals( + new MultipleSpecificSegmentSpec( + ImmutableList.of( + new SegmentDescriptor( + IDENTIFIERS.get(2).getInterval(), + IDENTIFIERS.get(2).getVersion(), + IDENTIFIERS.get(2).getShardSpec().getPartitionNum() + ) + ) + ) + ) + .build(); + + final List> results1 = Lists.newArrayList(); + Sequences.toList(query1.run(appenderator, ImmutableMap.of()), results1); + Assert.assertEquals( + "query1", + ImmutableList.of( + new Result<>( + new DateTime("2001"), + new TimeseriesResultValue(ImmutableMap.of("count", 2L, "met", 24L)) + ) + ), + results1 + ); + + // Query1: segment #2, partial + final TimeseriesQuery query2 = Druids.newTimeseriesQueryBuilder() + .dataSource(AppenderatorTester.DATASOURCE) + .aggregators( + Arrays.asList( + new LongSumAggregatorFactory("count", "count"), + new LongSumAggregatorFactory("met", "met") + ) + ) + .granularity(QueryGranularity.DAY) + .intervals( + new MultipleSpecificSegmentSpec( + ImmutableList.of( + new SegmentDescriptor( + new Interval("2001/PT1H"), + IDENTIFIERS.get(2).getVersion(), + IDENTIFIERS.get(2).getShardSpec().getPartitionNum() + ) + ) + ) + ) + .build(); + + final List> results2 = Lists.newArrayList(); + Sequences.toList(query2.run(appenderator, ImmutableMap.of()), results2); + Assert.assertEquals( + "query2", + ImmutableList.of( + new Result<>( + new DateTime("2001"), + new TimeseriesResultValue(ImmutableMap.of("count", 1L, "met", 8L)) + ) + ), + results2 + ); + } + } + + private static SegmentIdentifier SI(String interval, String version, int partitionNum) + { + return new SegmentIdentifier( + AppenderatorTester.DATASOURCE, + new Interval(interval), + version, + new LinearShardSpec(partitionNum) + ); + } + + private static InputRow IR(String ts, String dim, long met) + { + return new MapBasedInputRow( + new DateTime(ts).getMillis(), + ImmutableList.of("dim"), + ImmutableMap.of( + "dim", + dim, + "met", + met + ) + ); + } + + private static Supplier committerSupplierFromConcurrentMap(final ConcurrentMap map) + { + return new Supplier() + { + @Override + public Committer get() + { + final Map mapCopy = ImmutableMap.copyOf(map); + + return new Committer() + { + @Override + public Object getMetadata() + { + return mapCopy; + } + + @Override + public void run() + { + // Do nothing + } + }; + } + }; + } + + private static List sorted(final List xs) + { + final List xsSorted = Lists.newArrayList(xs); + Collections.sort( + xsSorted, new Comparator() + { + @Override + public int compare(T a, T b) + { + if (a instanceof SegmentIdentifier && b instanceof SegmentIdentifier) { + return ((SegmentIdentifier) a).getIdentifierAsString() + .compareTo(((SegmentIdentifier) b).getIdentifierAsString()); + } else if (a instanceof DataSegment && b instanceof DataSegment) { + return ((DataSegment) a).getIdentifier() + .compareTo(((DataSegment) b).getIdentifier()); + } else { + throw new IllegalStateException("WTF??"); + } + } + } + ); + return xsSorted; + } + +} diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java new file mode 100644 index 000000000000..49c21d8bbed8 --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -0,0 +1,274 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.Granularity; +import com.metamx.common.logger.Logger; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.concurrent.Execs; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; +import io.druid.query.IntervalChunkingQueryRunnerDecorator; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMaker; +import io.druid.segment.IndexMerger; +import io.druid.segment.column.ColumnConfig; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; + +public class AppenderatorTester implements AutoCloseable +{ + public static final String DATASOURCE = "foo"; + + private final DataSchema schema; + private final RealtimeTuningConfig tuningConfig; + private final FireDepartmentMetrics metrics; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final Appenderator appenderator; + private final ExecutorService queryExecutor; + private final IndexIO indexIO; + private final IndexMerger indexMerger; + private final IndexMaker indexMaker; + private final ServiceEmitter emitter; + + private final List pushedSegments = new CopyOnWriteArrayList<>(); + + public AppenderatorTester( + final int maxRowsInMemory + ) + { + this(maxRowsInMemory, null); + } + + public AppenderatorTester( + final int maxRowsInMemory, + final File basePersistDirectory + ) + { + objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(LinearShardSpec.class); + + final Map parserMap = objectMapper.convertValue( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(null, null, null) + ) + ), + Map.class + ); + schema = new DataSchema( + DATASOURCE, + parserMap, + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + }, + new UniformGranularitySpec(Granularity.MINUTE, QueryGranularity.NONE, null), + objectMapper + ); + + tuningConfig = new RealtimeTuningConfig( + maxRowsInMemory, + null, + null, + basePersistDirectory, + null, + null, + null, + null, + null + ); + + metrics = new FireDepartmentMetrics(); + queryExecutor = Execs.singleThreaded("queryExecutor(%d)"); + + indexIO = new IndexIO( + objectMapper, + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 0; + } + } + ); + indexMerger = new IndexMerger(objectMapper, indexIO); + indexMaker = new IndexMaker(objectMapper, indexIO); + + emitter = new ServiceEmitter( + "test", + "test", + new LoggingEmitter( + new Logger(AppenderatorTester.class), + LoggingEmitter.Level.INFO, + objectMapper + ) + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + dataSegmentPusher = new DataSegmentPusher() + { + @Override + public String getPathForHadoop(String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + pushedSegments.add(segment); + return segment; + } + }; + appenderator = Appenderators.realtime( + schema, + tuningConfig, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>of( + TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + new IntervalChunkingQueryRunnerDecorator( + queryExecutor, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + emitter + ) + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + ), + new DataSegmentAnnouncer() + { + @Override + public void announceSegment(DataSegment segment) throws IOException + { + + } + + @Override + public void unannounceSegment(DataSegment segment) throws IOException + { + + } + + @Override + public void announceSegments(Iterable segments) throws IOException + { + + } + + @Override + public void unannounceSegments(Iterable segments) throws IOException + { + + } + }, + emitter, + queryExecutor + ); + } + + public DataSchema getSchema() + { + return schema; + } + + public RealtimeTuningConfig getTuningConfig() + { + return tuningConfig; + } + + public FireDepartmentMetrics getMetrics() + { + return metrics; + } + + public DataSegmentPusher getDataSegmentPusher() + { + return dataSegmentPusher; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public Appenderator getAppenderator() + { + return appenderator; + } + + public List getPushedSegments() + { + return pushedSegments; + } + + @Override + public void close() throws Exception + { + appenderator.close(); + queryExecutor.shutdownNow(); + emitter.close(); + FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); + } +} diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/CommittedTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/CommittedTest.java new file mode 100644 index 000000000000..29c5f3ba4a3f --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/CommittedTest.java @@ -0,0 +1,109 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.timeline.partition.LinearShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class CommittedTest +{ + private static final ObjectMapper objectMapper = new DefaultObjectMapper(); + + private static final SegmentIdentifier IDENTIFIER_OBJECT1 = new SegmentIdentifier( + "foo", + new Interval("2000/2001"), + "2000", + new LinearShardSpec(1) + ); + + private static final SegmentIdentifier IDENTIFIER_OBJECT2 = new SegmentIdentifier( + "foo", + new Interval("2001/2002"), + "2001", + new LinearShardSpec(1) + ); + + private static final SegmentIdentifier IDENTIFIER_OBJECT3 = new SegmentIdentifier( + "foo", + new Interval("2001/2002"), + "2001", + new LinearShardSpec(2) + ); + + private static final String IDENTIFIER1 = IDENTIFIER_OBJECT1.getIdentifierAsString(); + private static final String IDENTIFIER2 = IDENTIFIER_OBJECT2.getIdentifierAsString(); + private static final String IDENTIFIER3 = IDENTIFIER_OBJECT3.getIdentifierAsString(); + + private static Committed fixedInstance() + { + final Map hydrants = Maps.newHashMap(); + hydrants.put(IDENTIFIER1, 3); + hydrants.put(IDENTIFIER2, 2); + return new Committed(hydrants, ImmutableMap.of("metadata", "foo")); + } + + @Test + public void testFactoryMethod() + { + final Committed committed = fixedInstance(); + final Committed committed2 = Committed.create( + ImmutableMap.of( + IDENTIFIER_OBJECT1, 3, + IDENTIFIER_OBJECT2, 2 + ), + ImmutableMap.of("metadata", "foo") + ); + Assert.assertEquals(committed, committed2); + } + + @Test + public void testSerde() throws Exception + { + final Committed committed = fixedInstance(); + final byte[] bytes = objectMapper.writeValueAsBytes(committed); + final Committed committed2 = objectMapper.readValue(bytes, Committed.class); + Assert.assertEquals("Round trip: overall", committed, committed2); + Assert.assertEquals("Round trip: metadata", committed.getMetadata(), committed2.getMetadata()); + Assert.assertEquals("Round trip: identifiers", committed.getHydrants().keySet(), committed2.getHydrants().keySet()); + } + + @Test + public void testGetCommittedHydrant() + { + Assert.assertEquals(3, fixedInstance().getCommittedHydrants(IDENTIFIER1)); + Assert.assertEquals(2, fixedInstance().getCommittedHydrants(IDENTIFIER2)); + Assert.assertEquals(0, fixedInstance().getCommittedHydrants(IDENTIFIER3)); + } + + @Test + public void testWithout() throws Exception + { + Assert.assertEquals(0, fixedInstance().without(IDENTIFIER1).getCommittedHydrants(IDENTIFIER1)); + Assert.assertEquals(2, fixedInstance().without(IDENTIFIER1).getCommittedHydrants(IDENTIFIER2)); + } +} From af24a6ee3d25ce72c0789c5622840c5100907fe0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 24 Aug 2015 20:22:01 -0700 Subject: [PATCH 3/7] Support for datasource-level metadata, including a new "dataSource" table. --- .../metadata/MetadataStorageConnector.java | 3 +- .../metadata/MetadataStorageTablesConfig.java | 12 +- .../postgresql/PostgreSQLConnectorTest.java | 15 +- .../MetadataStorageUpdaterJobSpec.java | 1 + .../updater/HadoopConverterJobTest.java | 1 + .../common/actions/SegmentInsertAction.java | 40 ++- .../io/druid/indexing/common/TestTask.java | 29 --- .../actions/SegmentInsertActionTest.java | 175 +++++++++++++ .../common/actions/TaskActionTestKit.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 19 +- ...TestIndexerMetadataStorageCoordinator.java | 17 ++ .../IndexerMetadataStorageCoordinator.java | 32 ++- .../IndexerSQLMetadataStorageCoordinator.java | 229 +++++++++++++++--- .../druid/metadata/SQLMetadataConnector.java | 86 +++++-- ...exerSQLMetadataStorageCoordinatorTest.java | 124 +++++++++- .../main/java/io/druid/cli/CreateTables.java | 1 + 16 files changed, 667 insertions(+), 118 deletions(-) delete mode 100644 indexing-service/src/test/java/io/druid/indexing/common/TestTask.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java b/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java index 497ff75f85a5..3d72fe3496cc 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageConnector.java @@ -31,7 +31,6 @@ Void insertOrUpdate( final byte[] value ) throws Exception; - byte[] lookup( final String tableName, final String keyColumn, @@ -39,6 +38,8 @@ byte[] lookup( final String key ); + void createDataSourceTable(); + void createPendingSegmentsTable(); void createSegmentTable(); diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java b/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java index 8552ea45826a..20c870a8e070 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageTablesConfig.java @@ -31,7 +31,7 @@ public class MetadataStorageTablesConfig { public static MetadataStorageTablesConfig fromBase(String base) { - return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null); + return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null); } public static final String TASK_ENTRY_TYPE = "task"; @@ -45,6 +45,9 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("base") private final String base; + @JsonProperty("dataSource") + private final String dataSourceTable; + @JsonProperty("pendingSegments") private final String pendingSegmentsTable; @@ -72,6 +75,7 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonCreator public MetadataStorageTablesConfig( @JsonProperty("base") String base, + @JsonProperty("dataSource") String dataSourceTable, @JsonProperty("pendingSegments") String pendingSegmentsTable, @JsonProperty("segments") String segmentsTable, @JsonProperty("rules") String rulesTable, @@ -83,6 +87,7 @@ public MetadataStorageTablesConfig( ) { this.base = (base == null) ? DEFAULT_BASE : base; + this.dataSourceTable = makeTableName(dataSourceTable, "dataSource"); this.pendingSegmentsTable = makeTableName(pendingSegmentsTable, "pendingSegments"); this.segmentsTable = makeTableName(segmentsTable, "segments"); this.rulesTable = makeTableName(rulesTable, "rules"); @@ -115,6 +120,11 @@ public String getBase() return base; } + public String getDataSourceTable() + { + return dataSourceTable; + } + public String getPendingSegmentsTable() { return pendingSegmentsTable; diff --git a/extensions/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index 21c7201748ef..2fda23303814 100644 --- a/extensions/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -35,7 +35,20 @@ public void testIsTransientException() throws Exception { PostgreSQLConnector connector = new PostgreSQLConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), - Suppliers.ofInstance(new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null)) + Suppliers.ofInstance( + new MetadataStorageTablesConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ) ); Assert.assertTrue(connector.isTransientException(new SQLException("bummer, connection problem", "08DIE"))); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java index 274021c40dae..cb2e8f2a927d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java @@ -87,6 +87,7 @@ public String getPassword() public MetadataStorageTablesConfig getMetadataStorageTablesConfig() { return new MetadataStorageTablesConfig( + null, null, segmentTable, null, diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 0a18725839f3..7281c441f949 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -206,6 +206,7 @@ public InputStream openStream() throws IOException ); metadataStorageTablesConfigSupplier = derbyConnectorRule.metadataTablesConfigSupplier(); connector = derbyConnectorRule.getConnector(); + try { connector.getDBI().withHandle( new HandleCallback() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java index 5566f4c43564..dc97670082c9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentInsertAction.java @@ -20,7 +20,6 @@ package io.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; @@ -42,15 +41,27 @@ */ public class SegmentInsertAction implements TaskAction> { - @JsonIgnore private final Set segments; + private final Object oldCommitMetadata; + private final Object newCommitMetadata; + + public SegmentInsertAction( + Set segments + ) + { + this(segments, null, null); + } @JsonCreator public SegmentInsertAction( - @JsonProperty("segments") Set segments + @JsonProperty("segments") Set segments, + @JsonProperty("oldCommitMetadata") Object oldCommitMetadata, + @JsonProperty("newCommitMetadata") Object newCommitMetadata ) { this.segments = ImmutableSet.copyOf(segments); + this.oldCommitMetadata = oldCommitMetadata; + this.newCommitMetadata = newCommitMetadata; } @JsonProperty @@ -59,6 +70,18 @@ public Set getSegments() return segments; } + @JsonProperty + public Object getOldCommitMetadata() + { + return oldCommitMetadata; + } + + @JsonProperty + public Object getNewCommitMetadata() + { + return newCommitMetadata; + } + public TypeReference> getReturnTypeReference() { return new TypeReference>() @@ -69,9 +92,18 @@ public TypeReference> getReturnTypeReference() @Override public Set perform(Task task, TaskActionToolbox toolbox) throws IOException { + // TODO: It's possible that we lose our locks after calling this. This should be OK if we're using commitMetadata. + // TODO: Although, of course, that's not always used... toolbox.verifyTaskLocks(task, segments); - final Set retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments(segments); + // TODO: I'm pretty sure the attempt at transactionality is foiled by: + // TODO: - a zombie task can clobber a good segment on deep storage + // TODO: - announceHistoricalSegments will silently do nothing if one already exists with the same id + final Set retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + segments, + oldCommitMetadata, + newCommitMetadata + ); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestTask.java deleted file mode 100644 index 63e1b9234eb0..000000000000 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestTask.java +++ /dev/null @@ -1,29 +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; - -import io.druid.indexing.common.task.Task; - -/** - */ -public interface TestTask extends Task -{ - public TaskStatus getStatus(); -} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java new file mode 100644 index 000000000000..fa36afc4038f --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -0,0 +1,175 @@ +/* + * 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.actions; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.hamcrest.CoreMatchers; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.skife.jdbi.v2.exceptions.CallbackFailedException; + +public class SegmentInsertActionTest +{ + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Rule + public TaskActionTestKit actionTestKit = new TaskActionTestKit(); + + private static final String DATA_SOURCE = "none"; + private static final Interval INTERVAL = new Interval("2020/2020T01"); + private static final String PARTY_YEAR = "1999"; + private static final String THE_DISTANT_FUTURE = "3000"; + + private static final DataSegment SEGMENT1 = new DataSegment( + DATA_SOURCE, + INTERVAL, + PARTY_YEAR, + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of(), + new LinearShardSpec(0), + 9, + 1024 + ); + + private static final DataSegment SEGMENT2 = new DataSegment( + DATA_SOURCE, + INTERVAL, + PARTY_YEAR, + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of(), + new LinearShardSpec(1), + 9, + 1024 + ); + + private static final DataSegment SEGMENT3 = new DataSegment( + DATA_SOURCE, + INTERVAL, + THE_DISTANT_FUTURE, + ImmutableMap.of(), + ImmutableList.of(), + ImmutableList.of(), + new LinearShardSpec(1), + 9, + 1024 + ); + + @Test + public void testSimple() throws Exception + { + final Task task = new NoopTask(null, 0, 0, null, null, null); + final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + action.perform(task, actionTestKit.getTaskActionToolbox()); + + Assert.assertEquals( + ImmutableSet.of(SEGMENT1, SEGMENT2), + ImmutableSet.copyOf( + actionTestKit.getMetadataStorageCoordinator() + .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL) + ) + ); + } + + @Test + public void testTransactional() throws Exception + { + final Task task = new NoopTask(null, 0, 0, null, null, null); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + + new SegmentInsertAction( + ImmutableSet.of(SEGMENT1), + null, + ImmutableList.of(1) + ).perform( + task, + actionTestKit.getTaskActionToolbox() + ); + + new SegmentInsertAction( + ImmutableSet.of(SEGMENT2), + ImmutableList.of(1), + ImmutableList.of(2) + ).perform( + task, + actionTestKit.getTaskActionToolbox() + ); + + Assert.assertEquals( + ImmutableSet.of(SEGMENT1, SEGMENT2), + ImmutableSet.copyOf( + actionTestKit.getMetadataStorageCoordinator() + .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL) + ) + ); + + Assert.assertEquals( + ImmutableList.of(2), + actionTestKit.getMetadataStorageCoordinator().getDataSourceMetadata(DATA_SOURCE) + ); + } + + @Test + public void testFailBadVersion() throws Exception + { + final Task task = new NoopTask(null, 0, 0, null, null, null); + final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); + action.perform(task, actionTestKit.getTaskActionToolbox()); + } + + @Test + public void testFailTransactional() throws Exception + { + final Task task = new NoopTask(null, 0, 0, null, null, null); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox().lock(task, new Interval(INTERVAL)); + + thrown.expect(CallbackFailedException.class); + thrown.expectMessage(CoreMatchers.containsString("dataSource metadata transaction check failed")); + + new SegmentInsertAction( + ImmutableSet.of(SEGMENT1), + ImmutableList.of(1), + ImmutableList.of(2) + ).perform( + task, + actionTestKit.getTaskActionToolbox() + ); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java index a4d86fd2575e..0178ecfaef5c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java @@ -93,6 +93,7 @@ public void before() metadataStorageCoordinator, new NoopServiceEmitter() ); + testDerbyConnector.createDataSourceTable(); testDerbyConnector.createPendingSegmentsTable(); testDerbyConnector.createSegmentTable(); testDerbyConnector.createRulesTable(); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 45cf6ae6b38a..2cf2ca4878b4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -38,9 +38,7 @@ import com.metamx.common.ISE; import com.metamx.common.guava.Comparators; import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceEventBuilder; import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; @@ -69,7 +67,6 @@ import io.druid.indexing.common.task.IndexTask; import io.druid.indexing.common.task.KillTask; import io.druid.indexing.common.task.RealtimeIndexTask; -import io.druid.indexing.common.task.RealtimeIndexTaskTest; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; import io.druid.indexing.overlord.config.TaskQueueConfig; @@ -100,6 +97,7 @@ import io.druid.segment.realtime.FireDepartmentTest; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -223,20 +221,7 @@ public Set announceHistoricalSegments(Set segments) private static ServiceEmitter newMockEmitter() { - return new ServiceEmitter(null, null, null) - { - @Override - public void emit(Event event) - { - - } - - @Override - public void emit(ServiceEventBuilder builder) - { - - } - }; + return new NoopServiceEmitter(); } private static InputRow IR(String dt, String dim1, String dim2, float met) diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index f94d0a028893..d458cfb3ed1d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -43,6 +43,12 @@ public TestIndexerMetadataStorageCoordinator() unusedSegments = Lists.newArrayList(); } + @Override + public Object getDataSourceMetadata(String dataSource) + { + throw new UnsupportedOperationException(); + } + @Override public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException { @@ -77,6 +83,17 @@ public Set announceHistoricalSegments(Set segments) return ImmutableSet.copyOf(added); } + @Override + public Set announceHistoricalSegments( + Set segments, + Object oldCommitMetadata, + Object newCommitMetadata + ) throws IOException + { + // Don't actually compare metadata, just do it! + return announceHistoricalSegments(segments); + } + @Override public SegmentIdentifier allocatePendingSegment( String dataSource, diff --git a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 98474e88a7b2..f7899e114524 100644 --- a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -41,7 +41,7 @@ public interface IndexerMetadataStorageCoordinator * * @throws IOException */ - public List getUsedSegmentsForInterval(final String dataSource, final Interval interval) + List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException; /** @@ -65,7 +65,7 @@ public List getUsedSegmentsForIntervals(final String dataSource, fi * * @return set of segments actually added */ - public Set announceHistoricalSegments(final Set segments) throws IOException; + Set announceHistoricalSegments(Set segments) throws IOException; /** * Allocate a new pending segment in the pending segments table. This segment identifier will never be given out @@ -93,9 +93,31 @@ SegmentIdentifier allocatePendingSegment( String maxVersion ) throws IOException; - public void updateSegmentMetadata(final Set segments) throws IOException; + /** + * Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments + * with identifiers already in the metadata storage will not be added). + *

+ * If newCommitMetadata is set, this insertion will be atomic with a compare-and-swap on dataSource commit metadata. + * + * @param segments set of segments to add, must all be from the same dataSource + * @param oldCommitMetadata old dataSource commit metadata that should be set before this insertion. If null, we'll + * expect there to not be any existing metadata. + * @param newCommitMetadata new dataSource commit metadata that should be set after this insertion. If null this + * insert will not involve dataSource metadata + * + * @return set of segments actually added + */ + Set announceHistoricalSegments( + Set segments, + Object oldCommitMetadata, + Object newCommitMetadata + ) throws IOException; + + Object getDataSourceMetadata(String dataSource); + + void updateSegmentMetadata(Set segments) throws IOException; - public void deleteSegments(final Set segments) throws IOException; + void deleteSegments(Set segments) throws IOException; /** * Get all segments which include ONLY data within the given interval and are not flagged as used. @@ -105,5 +127,5 @@ SegmentIdentifier allocatePendingSegment( * * @return DataSegments which include ONLY data within the requested interval and are not flagged as used. Data segments NOT returned here may include data in the interval */ - public List getUnusedSegmentsForInterval(final String dataSource, final Interval interval); + List getUnusedSegmentsForInterval(String dataSource, Interval interval); } diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 2235916a1052..4628fd7c556c 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -29,6 +29,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.lifecycle.LifecycleStart; @@ -259,11 +261,39 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi */ public Set announceHistoricalSegments(final Set segments) throws IOException { - return connector.getDBI().inTransaction( + return announceHistoricalSegments(segments, null, null); + } + + /** + * {@inheritDoc} + */ + @Override + public Set announceHistoricalSegments( + final Set segments, + final Object oldCommitMetadata, + final Object newCommitMetadata + ) throws IOException + { + // Segments must all be from the same dataSource. + if (segments.isEmpty()) { + throw new IllegalArgumentException("segment set must not be empty"); + } + + final String dataSource = segments.iterator().next().getDataSource(); + for (DataSegment segment : segments) { + if (!dataSource.equals(segment.getDataSource())) { + throw new IllegalArgumentException("segments must all be from the same dataSource"); + } + } + + return connector.retryTransaction( new TransactionCallback>() { @Override - public Set inTransaction(Handle handle, TransactionStatus transactionStatus) throws IOException + public Set inTransaction( + final Handle handle, + final TransactionStatus transactionStatus + ) throws Exception { final Set inserted = Sets.newHashSet(); @@ -273,6 +303,20 @@ public Set inTransaction(Handle handle, TransactionStatus transacti } } + if (newCommitMetadata != null) { + final boolean success = updateDataSourceMetadataWithHandle( + handle, + dataSource, + oldCommitMetadata, + newCommitMetadata + ); + + if (!success) { + // RuntimeException => no retries (there's no point; this will never succeed) + throw new RuntimeException("dataSource metadata transaction check failed"); + } + } + return ImmutableSet.copyOf(inserted); } } @@ -478,10 +522,10 @@ public SegmentIdentifier inTransaction(Handle handle, TransactionStatus transact } /** - * Attempts to insert a single segment to the database. If the segment already exists, will do nothing. Meant - * to be called from within a transaction. + * Attempts to insert a single segment to the database. If the segment already exists, will do nothing; although, + * this checking is imperfect and callers must be prepared to retry their entire transaction on exceptions. * - * @return true if the segment was added, false otherwise + * @return true if the segment was added, false if it already existed */ private boolean announceHistoricalSegment(final Handle handle, final DataSegment segment) throws IOException { @@ -491,38 +535,31 @@ private boolean announceHistoricalSegment(final Handle handle, final DataSegment return false; } - // Try/catch to work around races due to SELECT -> INSERT. Avoid ON DUPLICATE KEY since it's not portable. - try { - handle.createStatement( - String.format( - "INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - dbTables.getSegmentsTable() - ) - ) - .bind("id", segment.getIdentifier()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", new DateTime().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .execute(); - - log.info("Published segment [%s] to DB", segment.getIdentifier()); - } - catch (Exception e) { - if (e.getCause() instanceof SQLException && segmentExists(handle, segment)) { - log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); - } else { - throw e; - } - } + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + handle.createStatement( + String.format( + "INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + dbTables.getSegmentsTable() + ) + ) + .bind("id", segment.getIdentifier()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", new DateTime().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .execute(); + + log.info("Published segment [%s] to DB", segment.getIdentifier()); } - catch (IOException e) { - log.error(e, "Exception inserting into DB"); + catch (Exception e) { + log.error(e, "Exception inserting segment [%s] into DB", segment.getIdentifier()); throw e; } @@ -543,6 +580,126 @@ private boolean segmentExists(final Handle handle, final DataSegment segment) .isEmpty(); } + /** + * Read dataSource metadata. Returns null if there is no metadata. + */ + public Object getDataSourceMetadata(final String dataSource) + { + final byte[] bytes = connector.lookup( + dbTables.getDataSourceTable(), + "dataSource", + "commit_metadata_payload", + dataSource + ); + + try { + return jsonMapper.readValue(bytes, Object.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Read dataSource metadata as bytes, from a specific handle. Returns null if there is no metadata. + */ + private byte[] getDataSourceMetadataWithHandleAsBytes( + final Handle handle, + final String dataSource + ) + { + return connector.lookupWithHandle( + handle, + dbTables.getDataSourceTable(), + "dataSource", + "commit_metadata_payload", + dataSource + ); + } + + /** + * Compare-and-swap dataSource metadata in a transaction. + *

+ * Returns true if successful or false if nothing was actually modified because the oldCommitMetadata did not match. + * oldCommitMetadata can be null, in which case we'll expect there to be no metadata row existing for this dataSource. + */ + private boolean updateDataSourceMetadataWithHandle( + final Handle handle, + final String dataSource, + final T oldCommitMetadata, + final T newCommitMetadata + ) throws IOException + { + Preconditions.checkNotNull(dataSource, "dataSource"); + Preconditions.checkNotNull(newCommitMetadata, "newCommitMetadata"); + + final byte[] oldCommitMetadataBytesFromDb = getDataSourceMetadataWithHandleAsBytes(handle, dataSource); + final String oldCommitMetadataSha1FromDb; + if (oldCommitMetadataBytesFromDb == null) { + oldCommitMetadataSha1FromDb = null; + } else { + oldCommitMetadataSha1FromDb = BaseEncoding.base16().encode( + Hashing.sha1().hashBytes(oldCommitMetadataBytesFromDb).asBytes() + ); + } + + final byte[] newCommitMetadataBytes = jsonMapper.writeValueAsBytes(newCommitMetadata); + final String newCommitMetadataSha1 = BaseEncoding.base16().encode( + Hashing.sha1().hashBytes(newCommitMetadataBytes).asBytes() + ); + + if (oldCommitMetadata == null) { + // Expecting no old metadata; confirm it doesn't exist and then INSERT the new stuff. + if (oldCommitMetadataBytesFromDb != null) { + return false; + } + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + final int numRows = handle.createStatement( + String.format( + "INSERT INTO %s (dataSource, created_date, commit_metadata_payload, commit_metadata_sha1) " + + "VALUES (:dataSource, :created_date, :commit_metadata_payload, :commit_metadata_sha1)", + dbTables.getDataSourceTable() + ) + ) + .bind("dataSource", dataSource) + .bind("created_date", new DateTime().toString()) + .bind("commit_metadata_payload", newCommitMetadataBytes) + .bind("commit_metadata_sha1", newCommitMetadataSha1) + .execute(); + + return numRows > 0; + } else { + // Expecting a particular old metadata; confirm the bytes in the DB really do match what we want + if (oldCommitMetadataBytesFromDb == null) { + return false; + } + + final Object oldCommitMetadataFromDb = jsonMapper.readValue(oldCommitMetadataBytesFromDb, Object.class); + if (!oldCommitMetadataFromDb.equals(oldCommitMetadata)) { + return false; + } + + // The bytes matched, use the SHA1 in a compare-and-swap UPDATE + final int numRows = handle.createStatement( + String.format( + "UPDATE %s SET " + + "commit_metadata_payload = :new_commit_metadata_payload, " + + "commit_metadata_sha1 = :new_commit_metadata_sha1 " + + "WHERE dataSource = :dataSource AND commit_metadata_sha1 = :old_commit_metadata_sha1", + dbTables.getDataSourceTable() + ) + ) + .bind("dataSource", dataSource) + .bind("old_commit_metadata_sha1", oldCommitMetadataSha1FromDb) + .bind("new_commit_metadata_payload", newCommitMetadataBytes) + .bind("new_commit_metadata_sha1", newCommitMetadataSha1) + .execute(); + + return numRows > 0; + } + } + public void updateSegmentMetadata(final Set segments) throws IOException { connector.getDBI().inTransaction( diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java index 3846298ba3fa..655bf7418584 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java @@ -207,6 +207,25 @@ tableName, getPayloadType() ); } + public void createDataSourceTable(final String tableName) + { + createTable( + tableName, + ImmutableList.of( + String.format( + "CREATE TABLE %1$s (\n" + + " dataSource VARCHAR(255) NOT NULL,\n" + + " created_date VARCHAR(255) NOT NULL,\n" + + " commit_metadata_payload %2$s NOT NULL,\n" + + " commit_metadata_sha1 VARCHAR(255) NOT NULL,\n" + + " PRIMARY KEY (dataSource)\n" + + ")", + tableName, getPayloadType() + ) + ) + ); + } + public void createSegmentTable(final String tableName) { createTable( @@ -381,6 +400,13 @@ public Void inTransaction(Handle handle, TransactionStatus transactionStatus) th public abstract DBI getDBI(); + public void createDataSourceTable() + { + if (config.get().isCreateTables()) { + createDataSourceTable(tablesConfigSupplier.get().getDataSourceTable()); + } + } + @Override public void createPendingSegmentsTable() { @@ -398,21 +424,24 @@ public void createSegmentTable() } @Override - public void createRulesTable() { + public void createRulesTable() + { if (config.get().isCreateTables()) { createRulesTable(tablesConfigSupplier.get().getRulesTable()); } } @Override - public void createConfigTable() { + public void createConfigTable() + { if (config.get().isCreateTables()) { createConfigTable(tablesConfigSupplier.get().getConfigTable()); } } @Override - public void createTaskTables() { + public void createTaskTables() + { if (config.get().isCreateTables()) { final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get(); final String entryType = tablesConfig.getTaskEntryType(); @@ -430,34 +459,47 @@ public byte[] lookup( final String key ) { - final String selectStatement = String.format("SELECT %s FROM %s WHERE %s = :key", valueColumn, - tableName, keyColumn); - return getDBI().withHandle( new HandleCallback() { @Override public byte[] withHandle(Handle handle) throws Exception { - List matched = handle.createQuery(selectStatement) - .bind("key", key) - .map(ByteArrayMapper.FIRST) - .list(); - - if (matched.isEmpty()) { - return null; - } - - if (matched.size() > 1) { - throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key); - } - - return matched.get(0); + return lookupWithHandle(handle, tableName, keyColumn, valueColumn, key); } } ); } + public byte[] lookupWithHandle( + final Handle handle, + final String tableName, + final String keyColumn, + final String valueColumn, + final String key + ) + { + final String selectStatement = String.format( + "SELECT %s FROM %s WHERE %s = :key", valueColumn, + tableName, keyColumn + ); + + List matched = handle.createQuery(selectStatement) + .bind("key", key) + .map(ByteArrayMapper.FIRST) + .list(); + + if (matched.isEmpty()) { + return null; + } + + if (matched.size() > 1) { + throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key); + } + + return matched.get(0); + } + public MetadataStorageConnectorConfig getConfig() { return config.get(); } protected BasicDataSource getDatasource() @@ -500,8 +542,10 @@ tableName, getSerialType(), getPayloadType() ) ); } + @Override - public void createAuditTable() { + public void createAuditTable() + { if (config.get().isCreateTables()) { createAuditTable(tablesConfigSupplier.get().getAuditTable()); } diff --git a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 4da60a093cbd..21573e5a3632 100644 --- a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,11 +27,14 @@ import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; +import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.rules.ExpectedException; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; @@ -42,9 +45,13 @@ public class IndexerSQLMetadataStorageCoordinatorTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + @Rule + public ExpectedException thrown = ExpectedException.none(); + private final ObjectMapper mapper = new DefaultObjectMapper(); private final DataSegment defaultSegment = new DataSegment( - "dataSource", + "fooDataSource", Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), "version", ImmutableMap.of(), @@ -56,7 +63,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest ); private final DataSegment defaultSegment2 = new DataSegment( - "dataSource", + "fooDataSource", Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), "version", ImmutableMap.of(), @@ -88,6 +95,7 @@ public void setUp() { derbyConnector = derbyConnectorRule.getConnector(); mapper.registerSubtypes(LinearShardSpec.class); + derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); coordinator = new IndexerSQLMetadataStorageCoordinator( @@ -135,6 +143,113 @@ public void testSimpleAnnounce() throws IOException ); } + @Test + public void testTransactionalAnnounceSuccess() throws IOException + { + // Insert first segment. + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + null, + ImmutableMap.of("foo", "bar") + ); + + Assert.assertArrayEquals( + mapper.writeValueAsString(defaultSegment).getBytes("UTF-8"), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + defaultSegment.getIdentifier() + ) + ); + + // Insert second segment. + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment2), + ImmutableMap.of("foo", "bar"), + ImmutableMap.of("foo", "baz") + ); + + Assert.assertArrayEquals( + mapper.writeValueAsString(defaultSegment2).getBytes("UTF-8"), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + defaultSegment2.getIdentifier() + ) + ); + + // Examine metadata. + Assert.assertEquals( + ImmutableMap.of("foo", "baz"), + coordinator.getDataSourceMetadata("fooDataSource") + ); + } + + @Test + public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException + { + thrown.expectCause( + CoreMatchers.allOf( + CoreMatchers.instanceOf(RuntimeException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("dataSource metadata transaction check failed")) + ) + ); + + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + ImmutableMap.of("foo", "bar"), + ImmutableMap.of("foo", "baz") + ); + } + + @Test + public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException + { + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + null, + ImmutableMap.of("foo", "baz") + ); + + thrown.expectCause( + CoreMatchers.allOf( + CoreMatchers.instanceOf(RuntimeException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("dataSource metadata transaction check failed")) + ) + ); + + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment2), + null, + ImmutableMap.of("foo", "baz") + ); + } + + @Test + public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException + { + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + null, + ImmutableMap.of("foo", "baz") + ); + + thrown.expectCause( + CoreMatchers.allOf( + CoreMatchers.instanceOf(RuntimeException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("dataSource metadata transaction check failed")) + ) + ); + + coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment2), + ImmutableMap.of("foo", "qux"), + ImmutableMap.of("foo", "baz") + ); + } + @Test public void testSimpleUsedList() throws IOException { @@ -191,7 +306,10 @@ public void testMultiIntervalUsedList() throws IOException ImmutableList.of(defaultSegment3), coordinator.getUsedSegmentsForIntervals( defaultSegment.getDataSource(), - ImmutableList.of(Interval.parse("2015-01-03T00Z/2015-01-03T05Z"), Interval.parse("2015-01-03T09Z/2015-01-04T00Z")) + ImmutableList.of( + Interval.parse("2015-01-03T00Z/2015-01-03T05Z"), + Interval.parse("2015-01-03T09Z/2015-01-04T00Z") + ) ) ); } diff --git a/services/src/main/java/io/druid/cli/CreateTables.java b/services/src/main/java/io/druid/cli/CreateTables.java index 97091023c66d..c63b262bb9e6 100644 --- a/services/src/main/java/io/druid/cli/CreateTables.java +++ b/services/src/main/java/io/druid/cli/CreateTables.java @@ -108,6 +108,7 @@ public void run() { final Injector injector = makeInjector(); MetadataStorageConnector dbConnector = injector.getInstance(MetadataStorageConnector.class); + dbConnector.createDataSourceTable(); dbConnector.createPendingSegmentsTable(); dbConnector.createSegmentTable(); dbConnector.createRulesTable(); From bbd824a39f66539ba070eb4a12865c2a9cad6a19 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 13 Sep 2015 14:42:02 -0700 Subject: [PATCH 4/7] RealtimeIndexTaskTokyoDrift sketch. This is for push-based externally-coordinated ingestion, tranquility-style. --- .../task/RealtimeIndexTaskTokyoDrift.java | 1105 +++++++++++++++++ .../io/druid/indexing/common/task/Task.java | 1 + .../appenderator/ServerViewWatcher.java | 164 +++ 3 files changed, 1270 insertions(+) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/ServerViewWatcher.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java new file mode 100644 index 000000000000..fbed5a6611b9 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java @@ -0,0 +1,1105 @@ +/* + * 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.JsonProperty; +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.collect.Queues; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.common.Granularity; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.guava.Sequence; +import com.metamx.common.logger.Logger; +import io.druid.common.utils.JodaUtils; +import io.druid.concurrent.Execs; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.LockTryAcquireAction; +import io.druid.indexing.common.actions.SegmentAllocateAction; +import io.druid.indexing.common.actions.SegmentInsertAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.SegmentNotWritableException; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.segment.realtime.appenderator.ServerViewWatcher; +import io.druid.segment.realtime.firehose.ChatHandler; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.segment.realtime.plumber.Committers; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.timeline.DataSegment; +import org.eclipse.jetty.util.ConcurrentHashSet; +import org.joda.time.DateTime; + +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Random; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Push-based realtime task that uses Appenderators to write data. + */ +public class RealtimeIndexTaskTokyoDrift extends AbstractTask +{ + private static final String TYPE = "index_realtime_tokyo_drift"; + private static final Logger log = new Logger(RealtimeIndexTaskTokyoDrift.class); + private static final Random random = new Random(); + private static final long POLL_TIMEOUT = 100; + private static final int DEFAULT_BUFFER_SIZE = 100000; + private static final int MAX_SEGMENT_ROWS = 10000; + private static final int MAX_SEGMENTS = 5; + + private final InputRowParser> parser; + private final int slotNum; + private final DataSchema dataSchema; + private final RealtimeTuningConfig tuningConfig; + private final ChatHandlerProvider chatHandlerProvider; + private final ListeningExecutorService publishExecutor; + private final AtomicLong currentlyPublishing = new AtomicLong(0L); + private final Committer committer = Committers.nil(); + private final Supplier committerSupplier = Suppliers.ofInstance(committer); + private final RequestBuffer buffer; + + // Keys = Starts of segment intervals. Values = Active segments (currently adding data to). Only populated on leaders. + private final NavigableMap activeSegments = new TreeMap<>(); + + // SegmentIdentifiers we are currently trying to publish. + private final ConcurrentHashSet outgoingSegments = new ConcurrentHashSet<>(); + + // Next handoff is at this time. + private volatile long nextHandoffTime = JodaUtils.MAX_INSTANT; + + private volatile Appenderator appenderator = null; + private volatile long currentEpoch = 0; + private volatile boolean leading = false; + private volatile boolean stopped = false; + + @JsonCreator + public RealtimeIndexTaskTokyoDrift( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("maxBufferSize") Integer maxBufferSize, + @JsonProperty("slotNum") Integer slotNum, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") RealtimeTuningConfig tuningConfig, + @JsonProperty("context") Map context, + @JacksonInject ChatHandlerProvider chatHandlerProvider + ) + { + super( + id == null ? makeTaskId(dataSchema.getDataSource(), slotNum, random.nextInt()) : id, + String.format("%s_%s", TYPE, dataSchema.getDataSource()), + taskResource, + dataSchema.getDataSource(), + context + ); + + this.parser = Preconditions.checkNotNull((InputRowParser>) dataSchema.getParser(), "parser"); + this.buffer = new RequestBuffer(maxBufferSize == null ? DEFAULT_BUFFER_SIZE : maxBufferSize); + this.slotNum = Preconditions.checkNotNull(slotNum, "slotNum"); + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.chatHandlerProvider = chatHandlerProvider; + this.publishExecutor = MoreExecutors.listeningDecorator(Execs.newBlockingSingleThreaded(getId() + "[publish]", 1)); + } + + private static String makeTaskId(String dataSource, int slotNum, int randomBits) + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Ints.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((randomBits >>> (i * 4)) & 0x0F))); + } + return String.format( + "%s_%s_%d_%s", + TYPE, + dataSource, + slotNum, + suffix + ); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public String getNodeType() + { + return "realtime"; + } + + @Override + public QueryRunner getQueryRunner(Query query) + { + if (appenderator == null) { + return null; + } + + return new QueryRunner() + { + @Override + public Sequence run(final Query query, final Map responseContext) + { + return query.run(appenderator, responseContext); + } + }; + } + + @JsonProperty + public int getSlotNum() + { + return slotNum; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public RealtimeTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + log.info("Starting up!"); + + final FireDepartmentMetrics metrics = new FireDepartmentMetrics(); + final BlockingQueue handoffs = Queues.newLinkedBlockingQueue(); + try ( + Appenderator appenderator = newAppenderator(metrics, toolbox); + ServerViewWatcher serverViewWatcher = newServerViewWatcher(appenderator, toolbox, handoffs); + Resource resource = new Resource() + ) { + this.appenderator = appenderator; + + while (!stopped) { + final RttdRequest untypedRequest = buffer.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS); + + if (untypedRequest instanceof AppenderationRequest) { + final AppenderationRequest request = (AppenderationRequest) untypedRequest; + final AppenderationResponse response = handleAppenderationRequest(request, appenderator, metrics, toolbox); + request.getResponse().set(response); + } else if (untypedRequest instanceof ShutDownRequest) { + final ShutDownRequest request = (ShutDownRequest) untypedRequest; + final ShutDownResponse response = handleShutDownRequest(request); + request.getResponse().set(response); + } else if (untypedRequest != null) { + log.error("WTF?! Unrecognized request type."); + untypedRequest.getResponse().setException(new ISE("Unrecognized request type.")); + } + + moveElderSegmentsOut(); + publishOutgoingSegments(appenderator, toolbox); + dropPublishedSegments(appenderator, handoffs); + } + + // Stopped + // TODO: Push, wait for handoff of remaining segments + // TODO: Keep responding to requests after being told to shutDown; we might be a follower that needs a promotion to leader + RttdRequest request = null; + while ((request = buffer.poll()) != null) { + request.getResponse().setException(new ISE("Stopped")); + } + } + + return TaskStatus.success(getId()); + } + + public T performRequest(final RttdRequest request) throws InterruptedException + { + return buffer.perform(request); + } + + private AppenderationResponse handleAppenderationRequest( + final AppenderationRequest request, + final Appenderator appenderator, + final FireDepartmentMetrics metrics, + final TaskToolbox toolbox + ) throws IOException, SegmentNotWritableException + { + synchronized (activeSegments) { + if (request.getEpoch() < currentEpoch) { + return new AppenderationResponse(false, currentEpoch, ImmutableList.of()); + } else if (leading && (request.getEpoch() != currentEpoch || !request.isLeader())) { + log.warn( + "WTF?! Received request with isLeader[%s], epoch[%,d] but I was already leading with epoch[%,d]. Rejecting.", + request.isLeader(), + request.getEpoch(), + currentEpoch + ); + return new AppenderationResponse(false, currentEpoch, ImmutableList.of()); + } else if (!leading && request.isLeader() && request.getEpoch() == currentEpoch) { + log.warn( + "WTF?! Received request with isLeader[%s], epoch[%,d] but currentEpoch is already [%,d]. Rejecting.", + request.isLeader(), + request.getEpoch(), + currentEpoch + ); + return new AppenderationResponse(false, currentEpoch, ImmutableList.of()); + } else if (request.isLeader()) { + if (!leading) { + // TODO: Use dataSource metadata to ensure clean transition to a new epoch? Prevent old leader from publishing. + log.info("You have raised your voices in an unmistakable chorus."); + + assert request.getEpoch() > currentEpoch; + assert activeSegments.isEmpty(); + assert outgoingSegments.isEmpty(); + + currentEpoch = request.getEpoch(); + leading = true; + + // Acquire locks for all pending segments. + for (final SegmentIdentifier identifier : appenderator.getSegments()) { + final Optional tryLock = toolbox.getTaskActionClient() + .submit(new LockTryAcquireAction(identifier.getInterval())); + + if (tryLock.isPresent()) { + log.info("Taking over segment[%s] from prior epoch.", identifier); + outgoingSegments.add(identifier); + } else { + // TODO: Something better than dropping the segment. + // TODO: Priority locking would help here, so we can always get the lock. + log.warn("Could not reacquire lock for segment[%s], dropping.", identifier); + try { + appenderator.drop(identifier).get(); + } + catch (ExecutionException | InterruptedException e) { + throw Throwables.propagate(e); + } + } + } + + log.info( + "Leading slotNum[%,d], epoch[%,d] with %,d segments from prior epochs to manage.", + slotNum, + currentEpoch, + appenderator.getSegments().size() + ); + } else { + currentEpoch = request.getEpoch(); + } + + final List retVal = Lists.newArrayList(); + + for (SegmentAndRows segmentAndRows : request.getRows()) { + for (Map object : segmentAndRows.getRows()) { + final InputRow row; + + try { + row = parser.parse(object); + } + catch (Exception e) { + if (log.isDebugEnabled()) { + log.debug("Dropping unparseable row: %s", object); + } + metrics.incrementUnparseable(); + retVal.add(null); + continue; + } + + // The request won't have an identifier, so let's determine our own. + final SegmentIdentifier identifier = getSegment(toolbox.getTaskActionClient(), row.getTimestamp()); + + if (identifier != null) { + // Add the row, possibly publish the segment if it's full. + final int numRows = appenderator.add(identifier, row, committerSupplier); + if (numRows >= MAX_SEGMENT_ROWS) { + moveSegmentsOut(ImmutableList.of(identifier)); + } + + retVal.add(identifier); + } else { + if (log.isDebugEnabled()) { + log.debug("Dropping row: %s", object); + } + metrics.incrementThrownAway(); + retVal.add(null); + } + } + } + + if (retVal.size() != request.getRowCount()) { + throw new ISE("WTF?! Return size[%,d] does not match input size[%,d].", retVal.size(), request.getRowCount()); + } + + return new AppenderationResponse( + true, + currentEpoch, + retVal + ); + } else { + assert !leading; + currentEpoch = request.getEpoch(); + + final List retVal = Lists.newArrayList(); + + for (SegmentAndRows segmentAndRows : request.getRows()) { + for (Map object : segmentAndRows.getRows()) { + final InputRow row; + try { + row = Preconditions.checkNotNull(parser.parse(object), "row"); + } + catch (Exception e) { + if (log.isDebugEnabled()) { + log.debug("Dropping unparseable row: %s", object); + } + metrics.incrementUnparseable(); + retVal.add(null); + continue; + } + + // Use identifier from the request. + final SegmentIdentifier identifier = segmentAndRows.getIdentifier(); + + // Add without checking MAX_SEGMENT_ROWS; we're following the leader on this one. + appenderator.add(identifier, row, committerSupplier); + retVal.add(identifier); + } + } + + return new AppenderationResponse( + true, + currentEpoch, + retVal + ); + } + } + } + + private void moveElderSegmentsOut() + { + synchronized (activeSegments) { + // Publish any segments that should be handed off. + final long now = System.currentTimeMillis(); + if (now >= nextHandoffTime) { + final List toPublish = Lists.newArrayList(); + for (Map.Entry entry : activeSegments.entrySet()) { + if (now >= entry.getValue().getHandoffTime().getMillis()) { + toPublish.add(entry.getValue().getIdentifier()); + } + } + + moveSegmentsOut(toPublish); + } + } + } + + private void moveSegmentsOut(List identifiers) + { + synchronized (activeSegments) { + for (SegmentIdentifier identifier : identifiers) { + final long key = identifier.getInterval().getStartMillis(); + final ActiveSegment current = activeSegments.get(key); + if (current != null && current.getIdentifier().equals(identifier)) { + activeSegments.remove(key); + log.info("About to handoff segment[%s].", identifier); + outgoingSegments.add(identifier); + updateNextHandoffTime(); + } else { + log.info("Asked to handoff segment[%s], but it was not active. Skipping.", identifier); + } + } + } + } + + private void updateNextHandoffTime() + { + if (activeSegments.isEmpty()) { + nextHandoffTime = JodaUtils.MAX_INSTANT; + } else { + DateTime minHandoffTime = null; + for (ActiveSegment activeSegment : activeSegments.values()) { + if (minHandoffTime == null || activeSegment.getHandoffTime().isBefore(minHandoffTime)) { + minHandoffTime = activeSegment.getHandoffTime(); + } + } + nextHandoffTime = minHandoffTime.getMillis(); + } + } + + private void publishOutgoingSegments( + final Appenderator appenderator, + final TaskToolbox toolbox + ) + { + if (currentlyPublishing.get() == 0 && !outgoingSegments.isEmpty()) { + final List theList = ImmutableList.copyOf(outgoingSegments); + + currentlyPublishing.incrementAndGet(); + publishExecutor.submit( + new Runnable() + { + @Override + public void run() + { + try { + final SegmentsAndMetadata segmentsAndMetadata = appenderator.push(theList, committer).get(); + toolbox.getTaskActionClient().submit( + new SegmentInsertAction(ImmutableSet.copyOf(segmentsAndMetadata.getSegments())) + ); + for (SegmentIdentifier identifier : theList) { + outgoingSegments.remove(identifier); + } + } + catch (Exception e) { + log.warn(e, "Failed to publish outgoing segments. Will try again soon."); + } + finally { + currentlyPublishing.decrementAndGet(); + } + } + } + ); + } + } + + private void dropPublishedSegments( + final Appenderator appenderator, + final BlockingQueue handoffs + ) + { + // Try to drop one published segment. + SegmentIdentifier toDrop; + while ((toDrop = handoffs.poll()) != null) { + // TODO: This could fail, potentially, and then the segment will never get dropped. + appenderator.drop(toDrop); + } + } + + private ShutDownResponse handleShutDownRequest(final ShutDownRequest request) throws IOException + { + if (request.getEpoch() < currentEpoch) { + log.info( + "Received shutDown request for epoch[%,d] < currentEpoch[%,d], not shutting down.", + request.getEpoch(), + currentEpoch + ); + return new ShutDownResponse(false, currentEpoch); + } else { + log.info("Shutting down..."); + currentEpoch = request.getEpoch(); + stopped = true; + buffer.close(); + return new ShutDownResponse(true, currentEpoch); + } + } + + private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) + { + final Appenderator appenderator = Appenderators.realtime( + dataSchema, + tuningConfig.withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMerger(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryExecutorService() + ); + + log.info("We have an appenderator. Starting the job!"); + appenderator.startJob(); + + return appenderator; + } + + private ServerViewWatcher newServerViewWatcher( + final Appenderator appenderator, + final TaskToolbox toolbox, + final BlockingQueue handoffs + ) + { + ServerViewWatcher watcher = new ServerViewWatcher( + appenderator, + toolbox.getNewSegmentServerView(), + MoreExecutors.sameThreadExecutor(), + new ServerViewWatcher.Callback() + { + @Override + public void notify( + SegmentIdentifier pending, + DruidServerMetadata server, + DataSegment servedSegment + ) throws Exception + { + handoffs.put(pending); + } + } + ); + + watcher.start(); + return watcher; + } + + private SegmentIdentifier getSegment( + final TaskActionClient taskActionClient, + final DateTime timestamp + ) throws IOException + { + synchronized (activeSegments) { + final SegmentIdentifier identifier; + + // See if we already have an active segment for this timestamp. + final Map.Entry candidateEntry = activeSegments.floorEntry(timestamp.getMillis()); + if (candidateEntry != null && candidateEntry.getValue().getIdentifier().getInterval().contains(timestamp)) { + identifier = candidateEntry.getValue().getIdentifier(); + } else { + // Need a new segment. + + if (activeSegments.size() >= MAX_SEGMENTS) { + // Push eldest one to make room. + final SegmentIdentifier eldest = activeSegments.firstEntry().getValue().getIdentifier(); + moveSegmentsOut(ImmutableList.of(eldest)); + } + + // Allocate new segment. + // TODO: Release locks, ever. + identifier = allocateSegment(taskActionClient, timestamp); + + if (identifier != null) { + final DateTime handoffTime = computeHandoffTime( + identifier, + new DateTime(), + dataSchema.getGranularitySpec().getSegmentGranularity() + ); + + log.info("New segment[%s], handing off at[%s].", identifier, handoffTime); + final ActiveSegment holder = new ActiveSegment(identifier, handoffTime); + + if (activeSegments.containsKey(holder.getKey())) { + final ActiveSegment conflicting = activeSegments.get(holder.getKey()); + throw new ISE( + "WTF?! Allocated segment[%s] which conflicts with existing segment[%s].", + holder.getIdentifier().getIdentifierAsString(), + conflicting.getIdentifier().getIdentifierAsString() + ); + } + + activeSegments.put(holder.getKey(), holder); + updateNextHandoffTime(); + } else { + // Well, we tried. + // TODO: Blacklist the interval for a while, so we don't waste time continuously trying. + log.warn("Cannot allocate segment for timestamp[%s].", timestamp); + } + } + + return identifier; + } + } + + private SegmentIdentifier allocateSegment( + final TaskActionClient taskActionClient, + final DateTime timestamp + ) throws IOException + { + // Random sequenceName- we just want a new segment allocated. + final String sequenceName = UUID.randomUUID().toString(); + + return taskActionClient.submit( + new SegmentAllocateAction( + getDataSource(), + timestamp, + dataSchema.getGranularitySpec().getQueryGranularity(), + dataSchema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + null + ) + ); + } + + private DateTime computeHandoffTime( + final SegmentIdentifier identifier, + final DateTime creationTime, + final Granularity segmentGranularity + ) + { + return new DateTime( + Longs.max( + identifier.getInterval().getEnd().plus(tuningConfig.getWindowPeriod()).getMillis(), + segmentGranularity.bucket(creationTime).getEnd().plus(tuningConfig.getWindowPeriod()).getMillis() + ) + ); + } + + public class Resource implements ChatHandler, AutoCloseable + { + public Resource() + { + // TODO: Causes clutter in service discovery; want some better way of getting this info out of the overlord + chatHandlerProvider.register(getId(), this); + } + + @GET + @Path("/status") + @Produces(MediaType.APPLICATION_JSON) + public Map getStatus() + { + synchronized (activeSegments) { + return ImmutableMap.of( + "bufferSize", buffer.size(), + "activeSegments", Ordering.natural().sortedCopy( + Iterables.transform( + activeSegments.values(), + new Function() + { + @Override + public String apply(ActiveSegment activeSegment) + { + return activeSegment.getIdentifier().getIdentifierAsString(); + } + } + ) + ), + "allSegments", Ordering.natural().sortedCopy( + Iterables.transform( + appenderator.getSegments(), + new Function() + { + @Override + public String apply(SegmentIdentifier identifier) + { + return identifier.getIdentifierAsString(); + } + } + ) + ), + "currentEpoch", currentEpoch, + "leading", leading + ); + } + } + + @POST + @Path("/shutDown") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public ShutDownResponse shutDown(final ShutDownRequest request) throws InterruptedException + { + return performRequest(request); + } + + @POST + @Path("/appenderate") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public AppenderationResponse appenderate(final AppenderationRequest request) throws InterruptedException + { + // TODO: Server threads can all get blocked on .handle if the main loop is exerting backpressure + // TODO: Confirm nothing important is going to get blocked by that + return performRequest(request); + } + + @Override + public void close() + { + chatHandlerProvider.unregister(getId()); + } + } + + public static class ActiveSegment + { + private final SegmentIdentifier identifier; + private final DateTime handoffTime; + + public ActiveSegment(SegmentIdentifier identifier, DateTime handoffTime) + { + this.identifier = identifier; + this.handoffTime = handoffTime; + } + + public SegmentIdentifier getIdentifier() + { + return identifier; + } + + public DateTime getHandoffTime() + { + return handoffTime; + } + + public Long getKey() + { + return identifier.getInterval().getStartMillis(); + } + } + + public static class SegmentAndRows + { + private final SegmentIdentifier identifier; + private final List> rows; + + @JsonCreator + public SegmentAndRows( + @JsonProperty("identifier") SegmentIdentifier identifier, + @JsonProperty("rows") List> rows + ) + { + this.identifier = identifier; + this.rows = rows; + } + + @JsonProperty + public SegmentIdentifier getIdentifier() + { + return identifier; + } + + @JsonProperty + public List> getRows() + { + return rows; + } + } + + public interface RttdRequest + { + long getEpoch(); + + int getRowCount(); + + SettableFuture getResponse(); + } + + public static class ShutDownRequest implements RttdRequest + { + private final long epoch; + private final SettableFuture response; + + @JsonCreator + public ShutDownRequest(long epoch) + { + this.epoch = epoch; + this.response = SettableFuture.create(); + } + + @Override + @JsonProperty + public long getEpoch() + { + return epoch; + } + + @Override + public SettableFuture getResponse() + { + return response; + } + + @Override + public int getRowCount() + { + return 1; + } + } + + public static class ShutDownResponse + { + private final boolean success; + private final long epoch; + + @JsonCreator + public ShutDownResponse(boolean success, long epoch) + { + this.success = success; + this.epoch = epoch; + } + + @JsonProperty + public boolean isSuccess() + { + return success; + } + + @JsonProperty + public long getEpoch() + { + return epoch; + } + } + + /** + * Sent by event pushers when we are leading. + */ + public static class AppenderationRequest implements RttdRequest + { + private final long epoch; + private final boolean leader; + private final List rows; + private final SettableFuture response; + + @JsonCreator + public AppenderationRequest( + @JsonProperty("epoch") long epoch, + @JsonProperty("leader") boolean leader, + @JsonProperty("rows") List rows + ) + { + this.epoch = epoch; + this.leader = leader; + this.rows = rows; + this.response = SettableFuture.create(); + } + + @Override + @JsonProperty("epoch") + public long getEpoch() + { + return epoch; + } + + @JsonProperty("leader") + public boolean isLeader() + { + return leader; + } + + @JsonProperty("rows") + public List getRows() + { + return rows; + } + + @Override + public int getRowCount() + { + int rowCount = 0; + for (SegmentAndRows segmentAndRows : rows) { + rowCount += segmentAndRows.getRows().size(); + } + return rowCount; + } + + @Override + public SettableFuture getResponse() + { + return response; + } + } + + // TODO: Serialized form is too large + public static class AppenderationResponse + { + private final boolean success; + private final long epoch; + private final List segments; + + @JsonCreator + public AppenderationResponse( + @JsonProperty("success") boolean success, + @JsonProperty("epoch") long epoch, + @JsonProperty("segments") List segments + ) + { + this.success = success; + this.epoch = epoch; + this.segments = segments; + } + + @JsonProperty + public boolean isSuccess() + { + return success; + } + + @JsonProperty + public long getEpoch() + { + return epoch; + } + + @JsonProperty + public List getSegments() + { + return segments; + } + } + + public static class RequestBuffer implements Closeable + { + private final ReentrantLock lock = new ReentrantLock(); + private final Condition await = lock.newCondition(); + private final ArrayDeque deque = new ArrayDeque<>(); + private final int maxBufferSize; + + private volatile int bufferedMessages = 0; + private volatile boolean open = true; + + public RequestBuffer(int maxBufferSize) + { + this.maxBufferSize = maxBufferSize; + } + + public T perform(RttdRequest request) throws InterruptedException + { + put(request); + + // TODO: Bail out if buffer is closed while waiting for a resposne + try { + return request.getResponse().get(); + } + catch (Exception e) { + Throwables.propagateIfInstanceOf(e, InterruptedException.class); + throw Throwables.propagate(e); + } + } + + public void put(RttdRequest request) throws InterruptedException + { + Preconditions.checkNotNull(request, "request"); + + final int requestMessages = request.getRowCount(); + if (requestMessages > maxBufferSize) { + throw new IAE("Cannot add [%,d] messages with maxBufferSize[%,d].", requestMessages, maxBufferSize); + } + + lock.lockInterruptibly(); + try { + while (open && requestMessages + bufferedMessages > maxBufferSize) { + await.await(); + } + + if (!open) { + throw new ISE("Cannot add to closed buffer"); + } + + deque.add(request); + bufferedMessages += requestMessages; + } + finally { + lock.unlock(); + } + } + + public RttdRequest poll() + { + lock.lock(); + try { + return deque.poll(); + } + finally { + lock.unlock(); + } + } + + public RttdRequest poll(long timeout, TimeUnit unit) throws InterruptedException + { + long timeoutNanos = unit.toNanos(timeout); + + lock.lockInterruptibly(); + try { + while (deque.isEmpty()) { + if (timeoutNanos <= 0) { + return null; + } else { + timeoutNanos = await.awaitNanos(timeoutNanos); + } + } + + final RttdRequest retVal = Preconditions.checkNotNull(deque.poll(), "WTF?! Null poll?"); + bufferedMessages -= retVal.getRowCount(); + return retVal; + } + finally { + lock.unlock(); + } + } + + public int size() + { + return bufferedMessages; + } + + @Override + public void close() throws IOException + { + lock.lock(); + try { + open = false; + await.signalAll(); + } + finally { + lock.unlock(); + } + } + } +} 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 e8cf245e4e05..d2b1d1260d65 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 @@ -54,6 +54,7 @@ @JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), + @JsonSubTypes.Type(name = "index_realtime_tokyo_drift", value = RealtimeIndexTaskTokyoDrift.class), @JsonSubTypes.Type(name = "noop", value = NoopTask.class), @JsonSubTypes.Type(name = "version_converter", value = ConvertSegmentBackwardsCompatibleTask.class), // Backwards compat - Deprecated @JsonSubTypes.Type(name = "version_converter_sub", value = ConvertSegmentBackwardsCompatibleTask.SubTask.class), // backwards compat - Deprecated diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/ServerViewWatcher.java b/server/src/main/java/io/druid/segment/realtime/appenderator/ServerViewWatcher.java new file mode 100644 index 000000000000..6d1c24c8c450 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/ServerViewWatcher.java @@ -0,0 +1,164 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.google.api.client.repackaged.com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.metamx.common.logger.Logger; +import io.druid.client.FilteredServerView; +import io.druid.client.ServerView; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.timeline.DataSegment; + +import java.io.Closeable; +import java.util.concurrent.ExecutorService; + +/** + * Watches a FilteredServerView for pending segments either showing up, or being overshadowed, in the historical + * cluster. + */ +public class ServerViewWatcher implements Closeable +{ + private static final Logger log = new Logger(ServerViewWatcher.class); + + private final Appenderator appenderator; + private final FilteredServerView serverView; + private final ExecutorService serverViewExecutor; + private final Callback callback; + + private volatile boolean stopped = false; + + public ServerViewWatcher( + Appenderator appenderator, + FilteredServerView serverView, + ExecutorService serverViewExecutor, + Callback callback + ) + { + this.appenderator = Preconditions.checkNotNull(appenderator, "appenderator"); + this.serverView = Preconditions.checkNotNull(serverView, "serverView"); + this.serverViewExecutor = Preconditions.checkNotNull(serverViewExecutor, "serverViewExecutor"); + this.callback = Preconditions.checkNotNull(callback, "callback"); + } + + public void start() + { + serverView.registerSegmentCallback( + serverViewExecutor, + new ServerView.BaseSegmentCallback() + { + @Override + public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSegment servedSegment) + { + if (stopped) { + log.info("Unregistering ServerViewCallback"); + return ServerView.CallbackAction.UNREGISTER; + } + + if (!server.isAssignable()) { + return ServerView.CallbackAction.CONTINUE; + } + + final SegmentIdentifier served = SegmentIdentifier.fromDataSegment(servedSegment); + log.debug("Checking segment[%s] on server[%s]", served, server); + + if (appenderator.getDataSource().equals(servedSegment.getDataSource())) { + for (SegmentIdentifier pending : appenderator.getSegments()) { + // This isn't quite right, since we might unload some segments if they are only partially overshadowed + // or if their overshadowing chunk is not completely present in the timeline. Because we don't maintain + // a full timeline, let's err on the side of dropping things. + + if (pending.equals(served) || + (served.getInterval().overlaps(pending.getInterval()) + && served.getVersion().compareTo(pending.getVersion()) > 0)) { + log.info( + "Segment[%s] on server[%s] >= pending segment[%s].", + served, + server.getHost(), + pending + ); + + try { + callback.notify(pending, server, servedSegment); + } + catch (Exception e) { + log.warn( + e, "Callback failed for segment[%s] on server[%s] >= pending segment[%s].", + served, + server.getHost(), + pending + ); + } + } + } + } + + return ServerView.CallbackAction.CONTINUE; + } + }, + new Predicate() + { + @Override + public boolean apply(final DataSegment segment) + { + // Include any segments from our dataSource whose intervals overlap our pending segments. + + // Note that this filter's meaning changes over time (as the list of pending segments changes) and + // therefore we may miss some interesting segments if they showed up before we realized we should care + // about them. For now the workaround for this is to restart the ingester. + + return + appenderator.getDataSource().equals(segment.getDataSource()) && Iterables.any( + appenderator.getSegments(), + new Predicate() + { + @Override + public boolean apply(SegmentIdentifier identifier) + { + return segment.getInterval().overlaps(identifier.getInterval()); + } + } + ); + } + } + ); + } + + public void close() + { + stopped = true; + } + + public interface Callback + { + /** + * Called when the ServerViewWatcher notices that a particular pending identifier has either showed up, or + * has been overshadowed, in the historical cluster. + *

+ * May be called multiple times for the same segment. + * + * @param pending the pending identifier + * @param server the server we found a handed-off, or overshadowing, segment on + * @param servedSegment the handed-off, or overshadowing, segment we found + */ + void notify(SegmentIdentifier pending, DruidServerMetadata server, DataSegment servedSegment) throws Exception; + } +} From c8bf2cf0540edf2f8966c9931c18baf560a72bb7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 14 Aug 2015 10:01:14 -0700 Subject: [PATCH 5/7] The Realtimes and the Plumbers: Tokyo Drift --- .../realtime/plumber/PlumberSchool.java | 3 +- .../RealtimePlumberSchoolTokyoDrift.java | 132 +++++ .../plumber/RealtimePlumberTokyoDrift.java | 466 ++++++++++++++++++ 3 files changed, 600 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java create mode 100644 server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberTokyoDrift.java diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java index 8c49822b60d5..26a2d2f2edf4 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java @@ -30,7 +30,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RealtimePlumberSchool.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "realtime", value = RealtimePlumberSchool.class), - @JsonSubTypes.Type(name = "flushing", value = FlushingPlumberSchool.class) + @JsonSubTypes.Type(name = "flushing", value = FlushingPlumberSchool.class), + @JsonSubTypes.Type(name = "tokyoDrift", value = RealtimePlumberSchoolTokyoDrift.class) }) public interface PlumberSchool { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java new file mode 100644 index 000000000000..b47ea0fd49a6 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java @@ -0,0 +1,132 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.plumber; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.FilteredServerView; +import io.druid.guice.annotations.Processing; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMaker; +import io.druid.segment.IndexMerger; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.SegmentPublisher; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.timeline.partition.ShardSpec; + +import java.io.File; +import java.util.concurrent.ExecutorService; + +public class RealtimePlumberSchoolTokyoDrift implements PlumberSchool +{ + private final ServiceEmitter emitter; + private final QueryRunnerFactoryConglomerate conglomerate; + private final DataSegmentAnnouncer segmentAnnouncer; + private final ExecutorService queryExecutorService; + private final DataSegmentPusher dataSegmentPusher; + private final ObjectMapper objectMapper; + private final FilteredServerView serverView; + private final SegmentPublisher segmentPublisher; + private final IndexIO indexIO; + private final IndexMerger indexMerger; + private final IndexMaker indexMaker; + + public RealtimePlumberSchoolTokyoDrift( + @JacksonInject ServiceEmitter emitter, + @JacksonInject QueryRunnerFactoryConglomerate conglomerate, + @JacksonInject DataSegmentAnnouncer segmentAnnouncer, + @JacksonInject @Processing ExecutorService queryExecutorService, + @JacksonInject DataSegmentPusher dataSegmentPusher, + @JacksonInject ObjectMapper objectMapper, + @JacksonInject FilteredServerView serverView, + @JacksonInject SegmentPublisher segmentPublisher, + @JacksonInject IndexIO indexIO, + @JacksonInject IndexMerger indexMerger, + @JacksonInject IndexMaker indexMaker + ) + { + this.emitter = emitter; + this.conglomerate = conglomerate; + this.segmentAnnouncer = segmentAnnouncer; + this.queryExecutorService = queryExecutorService; + this.dataSegmentPusher = dataSegmentPusher; + this.objectMapper = objectMapper; + this.serverView = serverView; + this.segmentPublisher = segmentPublisher; + this.indexIO = indexIO; + this.indexMerger = indexMerger; + this.indexMaker = indexMaker; + } + + @Override + public Plumber findPlumber( + final DataSchema schema, + final RealtimeTuningConfig config, + final FireDepartmentMetrics metrics + ) + { + final RealtimeTuningConfig config2 = config.withBasePersistDirectory( + makeBasePersistSubdirectory( + config.getBasePersistDirectory(), + schema.getDataSource(), + config.getShardSpec() + ) + ); + + final Appenderator appenderator = Appenderators.realtime( + schema, + config2, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + conglomerate, + segmentAnnouncer, + emitter, + queryExecutorService + ); + + return new RealtimePlumberTokyoDrift( + appenderator, + schema, + config2, + serverView, + segmentPublisher + ); + } + + private static File makeBasePersistSubdirectory( + final File basePersistDirectory, + final String dataSource, + final ShardSpec shardSpec + ) + { + final File dataSourceDirectory = new File(basePersistDirectory, dataSource); + return new File(dataSourceDirectory, String.valueOf(shardSpec.getPartitionNum())); + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberTokyoDrift.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberTokyoDrift.java new file mode 100644 index 000000000000..78d1fcc724ef --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberTokyoDrift.java @@ -0,0 +1,466 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.plumber; + +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.metamx.common.Granularity; +import com.metamx.common.ISE; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.guava.Sequence; +import com.metamx.emitter.EmittingLogger; +import io.druid.client.FilteredServerView; +import io.druid.common.guava.ThreadRenamingCallable; +import io.druid.concurrent.Execs; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.segment.incremental.IndexSizeExceededException; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.realtime.SegmentPublisher; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.SegmentNotWritableException; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.segment.realtime.appenderator.ServerViewWatcher; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.joda.time.Period; + +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; + +public class RealtimePlumberTokyoDrift implements Plumber +{ + private static final EmittingLogger log = new EmittingLogger(RealtimePlumberTokyoDrift.class); + + private final Appenderator appenderator; + private final DataSchema schema; + private final RealtimeTuningConfig config; + private final RejectionPolicy rejectionPolicy; + private final FilteredServerView serverView; + private final SegmentPublisher segmentPublisher; + + private final Object handoffMonitor = new Object(); + private volatile ExecutorService serverViewExecutor = null; + private volatile ScheduledExecutorService scheduledExecutor = null; + private volatile ServerViewWatcher serverViewWatcher = null; + private volatile Supplier lastCommitterSupplier = null; + private volatile boolean cleanShutdown = true; + private volatile boolean shuttingDown = false; + private volatile boolean stopped = false; + + public RealtimePlumberTokyoDrift( + final Appenderator appenderator, + final DataSchema schema, + final RealtimeTuningConfig config, + final FilteredServerView serverView, + final SegmentPublisher segmentPublisher + ) + { + this.appenderator = appenderator; + this.schema = schema; + this.config = config.withBasePersistDirectory( + makeBasePersistSubdirectory( + config.getBasePersistDirectory(), + schema.getDataSource(), + config.getShardSpec() + ) + ); + this.rejectionPolicy = config.getRejectionPolicyFactory().create(config.getWindowPeriod()); + this.serverView = serverView; + this.segmentPublisher = segmentPublisher; + } + + private static File makeBasePersistSubdirectory( + final File basePersistDirectory, + final String dataSource, + final ShardSpec shardSpec + ) + { + final File dataSourceDirectory = new File(basePersistDirectory, dataSource); + return new File(dataSourceDirectory, String.valueOf(shardSpec.getPartitionNum())); + } + + @Override + public Object startJob() + { + final Object retVal = appenderator.startJob(); + initializeExecutors(); + startPersistThread(); + initializeServerViewWatcher(); + mergeAndPush(); + return retVal; + } + + @Override + public int add( + final InputRow row, + final Supplier committerSupplier + ) throws IndexSizeExceededException + { + if (!rejectionPolicy.accept(row.getTimestampFromEpoch())) { + return -1; + } + + final Interval segmentInterval = schema.getGranularitySpec() + .getSegmentGranularity() + .bucket(row.getTimestamp()); + + final SegmentIdentifier segmentIdentifier = new SegmentIdentifier( + appenderator.getDataSource(), + segmentInterval, + config.getVersioningPolicy().getVersion(segmentInterval), + config.getShardSpec() + ); + + try { + final int numRows = appenderator.add(segmentIdentifier, row, committerSupplier); + lastCommitterSupplier = committerSupplier; + return numRows; + } + catch (SegmentNotWritableException e) { + // Segment already started handoff + return -1; + } + } + + @Override + public QueryRunner getQueryRunner(Query query) + { + return new QueryRunner() + { + @Override + public Sequence run(final Query query, final Map responseContext) + { + return query.run(appenderator, responseContext); + } + }; + } + + @Override + public void persist(Committer committer) + { + appenderator.persistAll(committer); + } + + @Override + public void finishJob() + { + log.info("Shutting down..."); + + shuttingDown = true; + + List pending = appenderator.getSegments(); + if (pending.isEmpty()) { + log.info("No segments to hand off."); + } else { + log.info("Pushing segments: %s", Joiner.on(", ").join(pending)); + } + + try { + if (lastCommitterSupplier != null) { + // Push all remaining data + mergeAndPush(); + } + + synchronized (handoffMonitor) { + while (!pending.isEmpty()) { + log.info("Waiting to hand off: %s", Joiner.on(", ").join(pending)); + handoffMonitor.wait(); + pending = appenderator.getSegments(); + } + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + stopped = true; + shutdownServerViewWatcher(); + shutdownExecutors(); + appenderator.close(); + } + + if (!cleanShutdown) { + throw new ISE("Exception occurred during persist and merge."); + } + } + + private void initializeExecutors() + { + if (scheduledExecutor == null) { + scheduledExecutor = Execs.scheduledSingleThreaded("plumber_scheduled_%d"); + } + + if (serverViewExecutor == null) { + serverViewExecutor = Execs.singleThreaded("plumber_serverview_%d"); + } + } + + private void shutdownExecutors() + { + serverViewExecutor.shutdownNow(); + scheduledExecutor.shutdownNow(); + } + + private void startPersistThread() + { + final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); + final Period windowPeriod = config.getWindowPeriod(); + + final DateTime truncatedNow = segmentGranularity.truncate(new DateTime()); + final long windowMillis = windowPeriod.toStandardDuration().getMillis(); + + log.info( + "Expect to run at [%s]", + new DateTime().plus( + new Duration( + System.currentTimeMillis(), + segmentGranularity.increment(truncatedNow).getMillis() + windowMillis + ) + ) + ); + + ScheduledExecutors + .scheduleAtFixedRate( + scheduledExecutor, + new Duration( + System.currentTimeMillis(), + segmentGranularity.increment(truncatedNow).getMillis() + windowMillis + ), + new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)), + new ThreadRenamingCallable( + String.format( + "%s-overseer-%d", + schema.getDataSource(), + config.getShardSpec().getPartitionNum() + ) + ) + { + @Override + public ScheduledExecutors.Signal doCall() + { + if (stopped) { + log.info("Stopping merge-n-push overseer thread"); + return ScheduledExecutors.Signal.STOP; + } + + mergeAndPush(); + + if (stopped) { + log.info("Stopping merge-n-push overseer thread"); + return ScheduledExecutors.Signal.STOP; + } else { + return ScheduledExecutors.Signal.REPEAT; + } + } + } + ); + } + + private void mergeAndPush() + { + final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); + final Period windowPeriod = config.getWindowPeriod(); + + final long windowMillis = windowPeriod.toStandardDuration().getMillis(); + log.info("Starting merge and push."); + DateTime minTimestampAsDate = segmentGranularity.truncate( + new DateTime( + Math.max( + windowMillis, + rejectionPolicy.getCurrMaxTime() + .getMillis() + ) + - windowMillis + ) + ); + long minTimestamp = minTimestampAsDate.getMillis(); + + final List segments = appenderator.getSegments(); + final List segmentsToPush = Lists.newArrayList(); + + if (shuttingDown) { + log.info( + "Found [%,d] segments. Attempting to hand off all of them.", + segments.size() + ); + + segmentsToPush.addAll(segments); + } else { + log.info( + "Found [%,d] segments. Attempting to hand off segments that start before [%s].", + segments.size(), + minTimestampAsDate + ); + + for (SegmentIdentifier segment : segments) { + final Long intervalStart = segment.getInterval().getStartMillis(); + if (intervalStart < minTimestamp) { + log.info("Adding entry [%s] for merge and push.", segment); + segmentsToPush.add(segment); + } else { + log.info( + "Skipping persist and merge for entry [%s] : Start time [%s] >= [%s] min timestamp required in this run. Segment will be picked up in a future run.", + segment, + new DateTime(intervalStart), + minTimestampAsDate + ); + } + } + } + + log.info("Found [%,d] sinks to persist and merge", segmentsToPush.size()); + + // WARNING: Committers.nil() here means that on-disk data can get out of sync with committing + ListenableFuture publishFuture = Futures.transform( + appenderator.push(segmentsToPush, Committers.nil()), + new Function() + { + @Override + public Object apply(SegmentsAndMetadata pushedSegmentsAndMetadata) + { + // Immediately publish after pushing + for (DataSegment pushedSegment : pushedSegmentsAndMetadata.getSegments()) { + try { + segmentPublisher.publishSegment(pushedSegment); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + return null; + } + } + ); + + Futures.addCallback( + publishFuture, + new FutureCallback() + { + @Override + public void onSuccess(Object result) + { + log.info("Published [%,d] sinks.", segmentsToPush.size()); + } + + @Override + public void onFailure(Throwable e) + { + final List segmentIdentifierStrings = Lists.transform( + segmentsToPush, + new Function() + { + @Override + public String apply(SegmentIdentifier input) + { + return input.getIdentifierAsString(); + } + } + ); + + log.makeAlert(e, "Failed to publish merged indexes[%s]", schema.getDataSource()) + .addData("segments", segmentIdentifierStrings) + .emit(); + + if (shuttingDown) { + // We're trying to shut down, and these segments failed to push. Let's just get rid of them. + // This call will also delete possibly-partially-written files, so we don't need to do it explicitly. + cleanShutdown = false; + for (SegmentIdentifier identifier : segmentsToPush) { + dropSegment(identifier); + } + } + } + } + ); + } + + private void initializeServerViewWatcher() + { + if (serverViewWatcher == null) { + serverViewWatcher = new ServerViewWatcher( + appenderator, + serverView, + serverViewExecutor, + new ServerViewWatcher.Callback() + { + @Override + public void notify(SegmentIdentifier pending, DruidServerMetadata server, DataSegment servedSegment) + { + dropSegment(pending); + } + } + ); + + serverViewWatcher.start(); + } + } + + private void shutdownServerViewWatcher() + { + serverViewWatcher.close(); + } + + private void dropSegment(final SegmentIdentifier identifier) + { + log.info("Dropping segment: %s", identifier); + + Futures.addCallback( + appenderator.drop(identifier), + new FutureCallback() + { + @Override + public void onSuccess(Object result) + { + synchronized (handoffMonitor) { + handoffMonitor.notifyAll(); + } + } + + @Override + public void onFailure(Throwable e) + { + log.warn(e, "Failed to drop segment: %s", identifier); + synchronized (handoffMonitor) { + handoffMonitor.notifyAll(); + } + } + } + ); + } +} From d01414922a308d6badff6275b8d11feb0f042fd2 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 7 Dec 2015 12:18:00 -0800 Subject: [PATCH 6/7] fix rittd --- .../indexing/common/task/RealtimeIndexTaskTokyoDrift.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java index fbed5a6611b9..7e41576ec4ed 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; @@ -326,10 +325,10 @@ private AppenderationResponse handleAppenderationRequest( // Acquire locks for all pending segments. for (final SegmentIdentifier identifier : appenderator.getSegments()) { - final Optional tryLock = toolbox.getTaskActionClient() - .submit(new LockTryAcquireAction(identifier.getInterval())); + final TaskLock tryLock = toolbox.getTaskActionClient() + .submit(new LockTryAcquireAction(identifier.getInterval())); - if (tryLock.isPresent()) { + if (tryLock != null) { log.info("Taking over segment[%s] from prior epoch.", identifier); outgoingSegments.add(identifier); } else { From 556f27373d75289427ac70d5dd24a18f528ef0b4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 7 Dec 2015 13:05:06 -0800 Subject: [PATCH 7/7] appenderatorimpl updates based on rp changes --- .../task/RealtimeIndexTaskTokyoDrift.java | 4 +- .../appenderator/AppenderatorImpl.java | 73 +++++++++++++++---- .../realtime/appenderator/Appenderators.java | 14 +++- .../RealtimePlumberSchoolTokyoDrift.java | 16 ++-- .../appenderator/AppenderatorTester.java | 6 +- 5 files changed, 87 insertions(+), 26 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java index 7e41576ec4ed..0c51b58e9b11 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTaskTokyoDrift.java @@ -572,7 +572,9 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox.getQueryRunnerFactoryConglomerate(), toolbox.getSegmentAnnouncer(), toolbox.getEmitter(), - toolbox.getQueryExecutorService() + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig() ); log.info("We have an appenderator. Starting the job!"); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index ca25c67341eb..11c332ba0e02 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -41,6 +41,9 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.CachingQueryRunner; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.concurrent.Execs; import io.druid.data.input.Committer; @@ -63,6 +66,7 @@ import io.druid.segment.IndexSpec; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.Segment; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IndexSizeExceededException; @@ -107,6 +111,7 @@ public class AppenderatorImpl implements Appenderator private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class); private static final int WARN_DELAY = 1000; private static final String IDENTIFIER_FILE_NAME = "identifier.json"; + private static final String CONTEXT_SKIP_INCREMENTAL_SEGMENT = "skipIncrementalSegment"; private final DataSchema schema; private final RealtimeTuningConfig config; @@ -119,6 +124,8 @@ public class AppenderatorImpl implements Appenderator private final ExecutorService queryExecutorService; private final IndexIO indexIO; private final IndexMerger indexMerger; + private final Cache cache; + private final CacheConfig cacheConfig; private final Map sinks = Maps.newConcurrentMap(); private final Set droppingSinks = Sets.newConcurrentHashSet(); private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( @@ -142,7 +149,9 @@ public AppenderatorImpl( ServiceEmitter emitter, ExecutorService queryExecutorService, IndexIO indexIO, - IndexMerger indexMerger + IndexMerger indexMerger, + Cache cache, + CacheConfig cacheConfig ) { this.schema = Preconditions.checkNotNull(schema, "schema"); @@ -156,13 +165,21 @@ public AppenderatorImpl( this.queryExecutorService = queryExecutorService; this.indexIO = indexIO; this.indexMerger = indexMerger; + this.cache = cache; + this.cacheConfig = cacheConfig; - // If we're not querying (no conglomerate) then it's ok for the other query stuff to be null. - // But otherwise, we need them all. if (conglomerate != null) { + // If we're not querying (no conglomerate) then it's ok for the other query stuff to be null. + // But otherwise, we need them all. Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); Preconditions.checkNotNull(emitter, "emitter"); Preconditions.checkNotNull(queryExecutorService, "queryExecutorService"); + Preconditions.checkNotNull(cache, "cache"); + Preconditions.checkNotNull(cacheConfig, "cacheConfig"); + + if (!cache.isLocal()) { + log.error("Configured cache is not local, caching will not be enabled"); + } } log.info("Creating appenderator for dataSource[%s]", schema.getDataSource()); @@ -371,6 +388,7 @@ public ServiceMetricEvent.Builder apply(@Nullable Query input) return toolchest.makeMetricBuilder(query); } }; + final boolean skipIncrementalSegment = query.getContextValue(CONTEXT_SKIP_INCREMENTAL_SEGMENT, false); return toolchest.mergeResults( factory.mergeRunners( @@ -381,19 +399,19 @@ public ServiceMetricEvent.Builder apply(@Nullable Query input) new Function>() { @Override - public QueryRunner apply(final SegmentDescriptor spec) + public QueryRunner apply(final SegmentDescriptor descriptor) { final PartitionHolder holder = sinkTimeline.findEntry( - spec.getInterval(), - spec.getVersion() + descriptor.getInterval(), + descriptor.getVersion() ); if (holder == null) { - return new ReportTimelineMissingSegmentQueryRunner<>(spec); + return new ReportTimelineMissingSegmentQueryRunner<>(descriptor); } - final PartitionChunk chunk = holder.getChunk(spec.getPartitionNumber()); + final PartitionChunk chunk = holder.getChunk(descriptor.getPartitionNumber()); if (chunk == null) { - return new ReportTimelineMissingSegmentQueryRunner<>(spec); + return new ReportTimelineMissingSegmentQueryRunner<>(descriptor); } final Sink theSink = chunk.getObject(); @@ -404,7 +422,7 @@ public QueryRunner apply(final SegmentDescriptor spec) builderFn, new BySegmentQueryRunner( theSink.getSegment().getIdentifier(), - spec.getInterval().getStart(), + descriptor.getInterval().getStart(), factory.mergeRunners( MoreExecutors.sameThreadExecutor(), Iterables.transform( @@ -414,18 +432,36 @@ public QueryRunner apply(final SegmentDescriptor spec) @Override public QueryRunner apply(final FireHydrant hydrant) { - // TODO: Make sure this still works when hydrants are actually closed - return new ReferenceCountingSegmentQueryRunner<>( - factory, - hydrant.getSegment() - ); + if (skipIncrementalSegment && !hydrant.hasSwapped()) { + return new NoopQueryRunner<>(); + } + + final ReferenceCountingSegment segment = hydrant.getSegment(); + final boolean cacheable = + hydrant.hasSwapped() // only use caching if data is immutable + && cache.isLocal(); // hydrants may not be in sync between replicas, make sure cache is local + + if (cacheable) { + return new CachingQueryRunner<>( + makeHydrantCacheIdentifier(hydrant, segment), + descriptor, + objectMapper, + cache, + toolchest, + new ReferenceCountingSegmentQueryRunner<>(factory, segment), + MoreExecutors.sameThreadExecutor(), + cacheConfig + ); + } else { + return new ReferenceCountingSegmentQueryRunner<>(factory, segment); + } } } ) ) ) ).withWaitMeasuredFromNow(), - new SpecificSegmentSpec(spec) + new SpecificSegmentSpec(descriptor) ); } } @@ -1155,4 +1191,9 @@ private void removeDirectory(final File target) } } } + + private static String makeHydrantCacheIdentifier(FireHydrant input, Segment segment) + { + return segment.getIdentifier() + "_" + input.getCount(); + } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java index 8d2061487146..c03d7ef83f7e 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.IndexMaker; @@ -48,7 +50,9 @@ public static Appenderator realtime( QueryRunnerFactoryConglomerate conglomerate, DataSegmentAnnouncer segmentAnnouncer, ServiceEmitter emitter, - ExecutorService queryExecutorService + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig ) { return new AppenderatorImpl( @@ -62,7 +66,9 @@ public static Appenderator realtime( emitter, queryExecutorService, indexIO, - indexMerger + indexMerger, + cache, + cacheConfig ); } @@ -112,7 +118,9 @@ public void unannounceSegments(Iterable segments) throws IOExceptio null, null, indexIO, - indexMerger + indexMerger, + null, + null ); } } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java index b47ea0fd49a6..cad5de531464 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTokyoDrift.java @@ -23,10 +23,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; -import io.druid.segment.IndexMaker; import io.druid.segment.IndexMerger; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -53,7 +54,8 @@ public class RealtimePlumberSchoolTokyoDrift implements PlumberSchool private final SegmentPublisher segmentPublisher; private final IndexIO indexIO; private final IndexMerger indexMerger; - private final IndexMaker indexMaker; + private final Cache cache; + private final CacheConfig cacheConfig; public RealtimePlumberSchoolTokyoDrift( @JacksonInject ServiceEmitter emitter, @@ -66,7 +68,8 @@ public RealtimePlumberSchoolTokyoDrift( @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject IndexIO indexIO, @JacksonInject IndexMerger indexMerger, - @JacksonInject IndexMaker indexMaker + @JacksonInject Cache cache, + @JacksonInject CacheConfig cacheConfig ) { this.emitter = emitter; @@ -79,7 +82,8 @@ public RealtimePlumberSchoolTokyoDrift( this.segmentPublisher = segmentPublisher; this.indexIO = indexIO; this.indexMerger = indexMerger; - this.indexMaker = indexMaker; + this.cache = cache; + this.cacheConfig = cacheConfig; } @Override @@ -108,7 +112,9 @@ public Plumber findPlumber( conglomerate, segmentAnnouncer, emitter, - queryExecutorService + queryExecutorService, + cache, + cacheConfig ); return new RealtimePlumberTokyoDrift( diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 49c21d8bbed8..a9c7f25bb4c3 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -26,6 +26,8 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.LoggingEmitter; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.cache.CacheConfig; +import io.druid.client.cache.MapCache; import io.druid.concurrent.Execs; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; @@ -224,7 +226,9 @@ public void unannounceSegments(Iterable segments) throws IOExceptio } }, emitter, - queryExecutor + queryExecutor, + MapCache.create(2048), + new CacheConfig() ); }