From ad5ffdf483d5071378528a619b55ee504676b8b9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 15 Sep 2015 13:31:14 -0700 Subject: [PATCH 1/6] 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 2d3b0682dc85..fcc704fff8ae 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 @@ -311,7 +311,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())); final CountDownLatch doneSignal = new CountDownLatch(1); From 92c828f90410125ac17e3cf35b615189298243a2 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 6 Jan 2016 14:10:51 -0500 Subject: [PATCH 2/6] Make SegmentHandoffNotifier Closeable. --- .../io/druid/indexing/common/task/RealtimeIndexTaskTest.java | 2 +- .../java/io/druid/indexing/overlord/TaskLifecycleTest.java | 2 +- .../plumber/CoordinatorBasedSegmentHandoffNotifier.java | 2 +- .../io/druid/segment/realtime/plumber/RealtimePlumber.java | 2 +- .../segment/realtime/plumber/SegmentHandoffNotifier.java | 5 +++-- 5 files changed, 7 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 85eb0a90418b..933968d06465 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1005,7 +1005,7 @@ public void start() } @Override - public void stop() + public void close() { //Noop } 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 f40b7357b1e5..d4d335800a58 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 @@ -451,7 +451,7 @@ public void start() } @Override - public void stop() + public void close() { //Noop } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java b/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java index f27d8b8537ce..a90a17e84bb3 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java @@ -159,7 +159,7 @@ public boolean apply(DruidServerMetadata input) } @Override - public void stop() + public void close() { scheduledExecutor.shutdown(); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 9bc9d4ab14ed..238ec238edc1 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -651,7 +651,7 @@ public String apply(Sink input) } } - handoffNotifier.stop(); + handoffNotifier.close(); shutdownExecutors(); stopped = true; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java b/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java index fbccba57bcdc..f0c6c8636225 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/SegmentHandoffNotifier.java @@ -21,9 +21,10 @@ import io.druid.query.SegmentDescriptor; +import java.io.Closeable; import java.util.concurrent.Executor; -public interface SegmentHandoffNotifier +public interface SegmentHandoffNotifier extends Closeable { /** * register a handOffCallback to be called when segment handoff is complete. @@ -47,6 +48,6 @@ boolean registerSegmentHandoffCallback( /** * Perform any final processing and clean up after ourselves. */ - void stop(); + void close(); } From 08284fea626931dbacc794c62833116d1c690f24 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 6 Jan 2016 22:33:07 -0800 Subject: [PATCH 3/6] Publish test-jar for indexing-service. --- indexing-service/pom.xml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 0572df236006..893b5c138889 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -76,4 +76,20 @@ + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + From 3d2214377dad2476ad3a8a78f65932d808c9e654 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 1 Mar 2016 15:38:32 -0800 Subject: [PATCH 4/6] 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. FiniteAppenderatorDrivers help simplify indexing a finite stream of data. Also: - Sink: Ability to consider itself "finished" vs "still writable". - Sink: Ability to return the number of rows contained within the sink. --- .../ActionBasedSegmentAllocator.java | 65 + .../ActionBasedUsedSegmentChecker.java | 87 ++ .../common/index/YeOldePlumberSchool.java | 9 +- .../ActionBasedUsedSegmentCheckerTest.java | 127 ++ .../indexing/RealtimeTuningConfig.java | 3 +- .../realtime/appenderator/Appenderator.java | 150 +++ .../appenderator/AppenderatorConfig.java | 40 + .../appenderator/AppenderatorImpl.java | 1188 +++++++++++++++++ .../realtime/appenderator/Appenderators.java | 133 ++ .../realtime/appenderator/Committed.java | 119 ++ .../FiniteAppenderatorDriver.java | 553 ++++++++ .../FiniteAppenderatorDriverMetadata.java | 72 + .../appenderator/SegmentAllocator.java | 40 + .../SegmentNotWritableException.java | 28 + .../appenderator/SegmentsAndMetadata.java | 87 ++ .../TransactionalSegmentPublisher.java | 40 + .../appenderator/UsedSegmentChecker.java | 37 + .../realtime/plumber/RealtimePlumber.java | 19 +- .../druid/segment/realtime/plumber/Sink.java | 122 +- .../segment/realtime/RealtimeManagerTest.java | 18 +- .../appenderator/AppenderatorTest.java | 456 +++++++ .../appenderator/AppenderatorTester.java | 286 ++++ .../realtime/appenderator/CommittedTest.java | 109 ++ .../FiniteAppenderatorDriverTest.java | 296 ++++ .../plumber/RealtimePlumberSchoolTest.java | 18 +- .../segment/realtime/plumber/SinkTest.java | 9 +- 26 files changed, 4059 insertions(+), 52 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java 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/AppenderatorConfig.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/FiniteAppenderatorDriver.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.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/main/java/io/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java create mode 100644 server/src/main/java/io/druid/segment/realtime/appenderator/UsedSegmentChecker.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 create mode 100644 server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java new file mode 100644 index 000000000000..203363d79d46 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java @@ -0,0 +1,65 @@ +/* + * 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.appenderator; + +import io.druid.indexing.common.actions.SegmentAllocateAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.realtime.appenderator.SegmentAllocator; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import org.joda.time.DateTime; + +import java.io.IOException; + +public class ActionBasedSegmentAllocator implements SegmentAllocator +{ + private final TaskActionClient taskActionClient; + private final DataSchema dataSchema; + private final String sequenceName; + + public ActionBasedSegmentAllocator( + TaskActionClient taskActionClient, + DataSchema dataSchema, + String sequenceName + ) + { + this.taskActionClient = taskActionClient; + this.dataSchema = dataSchema; + this.sequenceName = sequenceName; + } + + @Override + public SegmentIdentifier allocate( + final DateTime timestamp, + final String previousSegmentId + ) throws IOException + { + return taskActionClient.submit( + new SegmentAllocateAction( + dataSchema.getDataSource(), + timestamp, + dataSchema.getGranularitySpec().getQueryGranularity(), + dataSchema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId + ) + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java new file mode 100644 index 000000000000..6ee38cbda848 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.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.indexing.appenderator; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import io.druid.common.utils.JodaUtils; +import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.UsedSegmentChecker; +import io.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker +{ + private final TaskActionClient taskActionClient; + + public ActionBasedUsedSegmentChecker(TaskActionClient taskActionClient) + { + this.taskActionClient = taskActionClient; + } + + @Override + public Set findUsedSegments(Set identifiers) throws IOException + { + // Group by dataSource + final Map> identifiersByDataSource = Maps.newTreeMap(); + for (SegmentIdentifier identifier : identifiers) { + if (!identifiersByDataSource.containsKey(identifier.getDataSource())) { + identifiersByDataSource.put(identifier.getDataSource(), Sets.newHashSet()); + } + identifiersByDataSource.get(identifier.getDataSource()).add(identifier); + } + + final Set retVal = Sets.newHashSet(); + + for (Map.Entry> entry : identifiersByDataSource.entrySet()) { + final List intervals = JodaUtils.condenseIntervals( + Iterables.transform(entry.getValue(), new Function() + { + @Override + public Interval apply(SegmentIdentifier input) + { + return input.getInterval(); + } + }) + ); + + final List usedSegmentsForIntervals = taskActionClient.submit( + new SegmentListUsedAction(entry.getKey(), null, intervals) + ); + + for (DataSegment segment : usedSegmentsForIntervals) { + if (identifiers.contains(SegmentIdentifier.fromDataSegment(segment))) { + retVal.add(segment); + } + } + } + + return retVal; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 27683dea5613..02d256c55d6d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -102,7 +102,14 @@ public Plumber findPlumber( ) { // There can be only one. - final Sink theSink = new Sink(interval, schema, config, version); + final Sink theSink = new Sink( + interval, + schema, + config.getShardSpec(), + version, + config.getMaxRowsInMemory(), + config.isReportParseExceptions() + ); // Temporary directory to hold spilled segments. final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getIdentifier()); diff --git a/indexing-service/src/test/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java b/indexing-service/src/test/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java new file mode 100644 index 000000000000..e011a22b6155 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java @@ -0,0 +1,127 @@ +/* + * 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.appenderator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.UsedSegmentChecker; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.junit.Assert; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.Test; + +import java.io.IOException; +import java.util.Set; + +public class ActionBasedUsedSegmentCheckerTest +{ + @Test + public void testBasic() throws IOException + { + final TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class); + EasyMock.expect( + taskActionClient.submit( + new SegmentListUsedAction("bar", null, ImmutableList.of(new Interval("2002/P1D"))) + ) + ).andReturn( + ImmutableList.of( + DataSegment.builder() + .dataSource("bar") + .interval(new Interval("2002/P1D")) + .shardSpec(new LinearShardSpec(0)) + .version("b") + .build(), + DataSegment.builder() + .dataSource("bar") + .interval(new Interval("2002/P1D")) + .shardSpec(new LinearShardSpec(1)) + .version("b") + .build() + ) + ); + EasyMock.expect( + taskActionClient.submit( + new SegmentListUsedAction("foo", null, ImmutableList.of(new Interval("2000/P1D"), new Interval("2001/P1D"))) + ) + ).andReturn( + ImmutableList.of( + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2000/P1D")) + .shardSpec(new LinearShardSpec(0)) + .version("a") + .build(), + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2000/P1D")) + .shardSpec(new LinearShardSpec(1)) + .version("a") + .build(), + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2001/P1D")) + .shardSpec(new LinearShardSpec(1)) + .version("b") + .build(), + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2002/P1D")) + .shardSpec(new LinearShardSpec(1)) + .version("b") + .build() + ) + ); + EasyMock.replay(taskActionClient); + + final UsedSegmentChecker checker = new ActionBasedUsedSegmentChecker(taskActionClient); + final Set segments = checker.findUsedSegments( + ImmutableSet.of( + new SegmentIdentifier("foo", new Interval("2000/P1D"), "a", new LinearShardSpec(1)), + new SegmentIdentifier("foo", new Interval("2001/P1D"), "b", new LinearShardSpec(0)), + new SegmentIdentifier("bar", new Interval("2002/P1D"), "b", new LinearShardSpec(0)) + ) + ); + + Assert.assertEquals( + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2000/P1D")) + .shardSpec(new LinearShardSpec(1)) + .version("a") + .build(), + DataSegment.builder() + .dataSource("bar") + .interval(new Interval("2002/P1D")) + .shardSpec(new LinearShardSpec(0)) + .version("b") + .build() + ), + segments + ); + + EasyMock.verify(taskActionClient); + } +} 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 8e62d2a9a6d0..a636d51107ab 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.io.Files; import io.druid.segment.IndexSpec; +import io.druid.segment.realtime.appenderator.AppenderatorConfig; import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; import io.druid.segment.realtime.plumber.RejectionPolicyFactory; import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; @@ -36,7 +37,7 @@ /** */ -public class RealtimeTuningConfig implements TuningConfig +public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig { private static final int defaultMaxRowsInMemory = 75000; private static final Period defaultIntermediatePersistPeriod = new Period("PT10M"); 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..7d8e3d60fa2f --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderator.java @@ -0,0 +1,150 @@ +/* + * 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; + +/** + * An Appenderator indexes 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 does not decide which segments data should go into. + * It also doesn't publish segments to the metadata store or monitor handoff; 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, and push methods should all be called from the same thread. + * + * @param identifier the segment into which this row should be added + * @param row the row to add + * @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 active 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, and push methods should all be called from the same thread. + * + * @param committer a committer associated with all data that has been added so far + * + * @return future that resolves when all pending data has been persisted, contains commit metadata for this persist + */ + ListenableFuture persistAll(Committer committer); + + /** + * 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, and push methods should all be called from the same thread. + * + * @param identifiers list of segments to push + * @param committer a committer associated with all data that has been added so far + * + * @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/AppenderatorConfig.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java new file mode 100644 index 000000000000..4b2c99f644c0 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -0,0 +1,40 @@ +/* + * 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 io.druid.segment.IndexSpec; +import org.joda.time.Period; + +import java.io.File; + +public interface AppenderatorConfig +{ + boolean isReportParseExceptions(); + + int getMaxRowsInMemory(); + + int getMaxPendingPersists(); + + Period getIntermediatePersistPeriod(); + + IndexSpec getIndexSpec(); + + File getBasePersistDirectory(); +} 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..087a373a510f --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -0,0 +1,1188 @@ +/* + * 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.CloseQuietly; +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.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; +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.QueryRunnerHelper; +import io.druid.query.QueryToolChest; +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.IndexSizeExceededException; +import io.druid.segment.indexing.DataSchema; +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 org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.Closeable; +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 static final String CONTEXT_SKIP_INCREMENTAL_SEGMENT = "skipIncrementalSegment"; + + private final DataSchema schema; + private final AppenderatorConfig tuningConfig; + 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 Cache cache; + private final CacheConfig cacheConfig; + 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, + AppenderatorConfig tuningConfig, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + IndexIO indexIO, + IndexMerger indexMerger, + Cache cache, + CacheConfig cacheConfig + ) + { + this.schema = Preconditions.checkNotNull(schema, "schema"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + 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; + this.cache = cache; + this.cacheConfig = cacheConfig; + + 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()); + } + + @Override + public String getDataSource() + { + return schema.getDataSource(); + } + + @Override + public Object startJob() + { + tuningConfig.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 = getOrCreateSink(identifier); + int sinkRetVal; + + try { + sinkRetVal = sink.add(row); + } + catch (IndexSizeExceededException e) { + // Try one more time after swapping, then throw the exception out if it happens again. + persistAll(committerSupplier.get()); + sinkRetVal = sink.add(row); + } + + if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush) { + persistAll(committerSupplier.get()); + } + + if (sinkRetVal < 0) { + throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier); + } else { + return sink.getNumRows(); + } + } + + @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); + } else { + return sink.getNumRows(); + } + } + + private Sink getOrCreateSink(final SegmentIdentifier identifier) + { + Sink retVal = sinks.get(identifier); + + if (retVal == null) { + retVal = new Sink( + identifier.getInterval(), + schema, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + ); + + 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) { + throw new ISE("Unknown query type[%s].", query.getClass()); + } + + 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); + } + }; + final boolean skipIncrementalSegment = query.getContextValue(CONTEXT_SKIP_INCREMENTAL_SEGMENT, false); + + return toolchest.mergeResults( + factory.mergeRunners( + queryExecutorService, + FunctionalIterable + .create(specs) + .transform( + new Function>() + { + @Override + public QueryRunner apply(final SegmentDescriptor descriptor) + { + final PartitionHolder holder = sinkTimeline.findEntry( + descriptor.getInterval(), + descriptor.getVersion() + ); + if (holder == null) { + return new ReportTimelineMissingSegmentQueryRunner<>(descriptor); + } + + final PartitionChunk chunk = holder.getChunk(descriptor.getPartitionNumber()); + if (chunk == null) { + return new ReportTimelineMissingSegmentQueryRunner<>(descriptor); + } + + final Sink theSink = chunk.getObject(); + + return new SpecificSegmentQueryRunner<>( + new MetricsEmittingQueryRunner<>( + emitter, + builderFn, + new BySegmentQueryRunner( + theSink.getSegment().getIdentifier(), + descriptor.getInterval().getStart(), + factory.mergeRunners( + MoreExecutors.sameThreadExecutor(), + Iterables.transform( + theSink, + new Function>() + { + @Override + public QueryRunner apply(final FireHydrant hydrant) + { + if (skipIncrementalSegment && !hydrant.hasSwapped()) { + return new NoopQueryRunner<>(); + } + + // Prevent the underlying segment from swapping when its being iterated + final Pair segment = hydrant.getAndIncrementSegment(); + try { + QueryRunner baseRunner = QueryRunnerHelper.makeClosingQueryRunner( + factory.createRunner(segment.lhs), + segment.rhs + ); + + if (hydrant.hasSwapped() // only use caching if data is immutable + && cache.isLocal() // hydrants may not be in sync between replicas, make sure cache is local + ) { + return new CachingQueryRunner<>( + makeHydrantCacheIdentifier(hydrant, segment.lhs), + descriptor, + objectMapper, + cache, + toolchest, + baseRunner, + MoreExecutors.sameThreadExecutor(), + cacheConfig + ); + } else { + return baseRunner; + } + } + catch (RuntimeException e) { + CloseQuietly.close(segment.rhs); + throw e; + } + } + } + ) + ) + ) + ).withWaitMeasuredFromNow(), + new SpecificSegmentSpec(descriptor) + ); + } + } + ) + ) + ); + } + + @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 Object commitMetadata = committer.getMetadata(); + 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].", commitMetadata, 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, commitMetadata)); + + return commitMetadata; + } + 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 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. + log.info("Segment[%s] already pushed.", identifier); + 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, + tuningConfig.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 = tuningConfig.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(tuningConfig.getIntermediatePersistPeriod()).getMillis(); + } + + /** + * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer. + * + * @return persisted commit metadata + */ + private Object bootstrapSinksFromDisk() + { + Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!"); + + final File baseDir = tuningConfig.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 from[%s]: %s", baseDir, 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, + identifier.getShardSpec(), + identifier.getVersion(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions(), + hydrants + ); + sinks.put(identifier, currSink); + sinkTimeline.add( + currSink.getInterval(), + currSink.getVersion(), + identifier.getShardSpec().createChunk(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(), + identifier.getShardSpec().createChunk(sink) + ); + + if (removeOnDiskData) { + removeDirectory(computePersistDir(identifier)); + } + + return null; + } + }, + persistExecutor + ); + } + + private File computeCommitFile() + { + return new File(tuningConfig.getBasePersistDirectory(), "commit.json"); + } + + private File computeLockFile() + { + return new File(tuningConfig.getBasePersistDirectory(), ".lock"); + } + + private File computePersistDir(SegmentIdentifier identifier) + { + return new File(tuningConfig.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 = tuningConfig.getIndexSpec(); + persistedFile = indexMerger.persist( + indexToPersist.getIndex(), + identifier.getInterval(), + new File(persistDir, String.valueOf(indexToPersist.getCount())), + 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(); + } + } + } + + 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 new file mode 100644 index 000000000000..7ec9e566e07c --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/Appenderators.java @@ -0,0 +1,133 @@ +/* + * 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.client.cache.Cache; +import io.druid.client.cache.CacheConfig; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.IndexIO; +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 createRealtime( + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig + ) + { + return new AppenderatorImpl( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + conglomerate, + segmentAnnouncer, + emitter, + queryExecutorService, + indexIO, + indexMerger, + cache, + cacheConfig + ); + } + + public static Appenderator createOffline( + DataSchema schema, + AppenderatorConfig 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 + } + + @Override + public boolean isAnnounced(DataSegment segment) + { + return false; + } + }, + null, + null, + indexIO, + indexMerger, + null, + null + ); + } + + +} 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/FiniteAppenderatorDriver.java b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java new file mode 100644 index 000000000000..bda0d09f217a --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriver.java @@ -0,0 +1,553 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.query.SegmentDescriptor; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.timeline.DataSegment; +import org.joda.time.DateTime; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; + +/** + * A FiniteAppenderatorDriver drives an Appenderator to index a finite stream of data. This class does not help you + * index unbounded streams. All handoff is done at the end of indexing. + * + * This class helps with doing things that Appenderators don't, including deciding which segments to use (with a + * SegmentAllocator), publishing segments to the metadata store (with a SegmentPublisher), and monitoring handoff (with + * a SegmentHandoffNotifier). + * + * Note that the commit metadata stored by this class via the underlying Appenderator is not the same metadata as + * you pass in. It's wrapped in some extra metadata needed by the driver. + */ +public class FiniteAppenderatorDriver implements Closeable +{ + private static final Logger log = new Logger(FiniteAppenderatorDriver.class); + + private final Appenderator appenderator; + private final SegmentAllocator segmentAllocator; + private final SegmentHandoffNotifier handoffNotifier; + private final UsedSegmentChecker usedSegmentChecker; + private final ObjectMapper objectMapper; + private final int maxRowsPerSegment; + private final long handoffConditionTimeout; + + // Key = Start of segment intervals. Value = Segment we're currently adding data to. + // All access to "activeSegments" and "lastSegmentId" must be synchronized on "activeSegments". + private final NavigableMap activeSegments = new TreeMap<>(); + private volatile String lastSegmentId = null; + + // Notified when segments are dropped. + private final Object handoffMonitor = new Object(); + + /** + * Create a driver. + * + * @param appenderator appenderator + * @param segmentAllocator segment allocator + * @param handoffNotifierFactory handoff notifier factory + * @param usedSegmentChecker used segment checker + * @param objectMapper object mapper, used for serde of commit metadata + * @param maxRowsPerSegment maximum number of rows allowed in an entire segment (not a single persist) + * @param handoffConditionTimeout maximum number of millis allowed for handoff (not counting push/publish), zero + * means wait forever. + */ + public FiniteAppenderatorDriver( + Appenderator appenderator, + SegmentAllocator segmentAllocator, + SegmentHandoffNotifierFactory handoffNotifierFactory, + UsedSegmentChecker usedSegmentChecker, + ObjectMapper objectMapper, + int maxRowsPerSegment, + long handoffConditionTimeout + ) + { + this.appenderator = Preconditions.checkNotNull(appenderator, "appenderator"); + this.segmentAllocator = Preconditions.checkNotNull(segmentAllocator, "segmentAllocator"); + this.handoffNotifier = Preconditions.checkNotNull(handoffNotifierFactory, "handoffNotifierFactory") + .createSegmentHandoffNotifier(appenderator.getDataSource()); + this.usedSegmentChecker = Preconditions.checkNotNull(usedSegmentChecker, "usedSegmentChecker"); + this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); + this.maxRowsPerSegment = maxRowsPerSegment; + this.handoffConditionTimeout = handoffConditionTimeout; + } + + /** + * Perform any initial setup and return currently persisted commit metadata. + * + * Note that this method returns the same metadata you've passed in with your Committers, even though this class + * stores extra metadata on disk. + * + * @return currently persisted commit metadata + */ + public Object startJob() + { + handoffNotifier.start(); + + final FiniteAppenderatorDriverMetadata metadata = objectMapper.convertValue( + appenderator.startJob(), + FiniteAppenderatorDriverMetadata.class + ); + + log.info("Restored metadata[%s].", metadata); + + if (metadata != null) { + synchronized (activeSegments) { + for (SegmentIdentifier identifier : metadata.getActiveSegments()) { + activeSegments.put(identifier.getInterval().getStartMillis(), identifier); + } + lastSegmentId = metadata.getPreviousSegmentId(); + } + + return metadata.getCallerMetadata(); + } else { + return null; + } + } + + /** + * Clears out all our state and also calls {@link Appenderator#clear()} on the underlying Appenderator. + */ + public void clear() throws InterruptedException + { + synchronized (activeSegments) { + activeSegments.clear(); + } + appenderator.clear(); + } + + /** + * Add a row. Must not be called concurrently from multiple threads. + * + * @param row the row to add + * @param committerSupplier supplier of a committer associated with all data that has been added, including this row + * + * @return segment to which this row was added, or null if segment allocator returned null for this row + * + * @throws IOException if there is an I/O error while allocating or writing to a segment + */ + public SegmentIdentifier add( + final InputRow row, + final Supplier committerSupplier + ) throws IOException + { + final SegmentIdentifier identifier = getSegment(row.getTimestamp()); + + if (identifier != null) { + try { + final int numRows = appenderator.add(identifier, row, wrapCommitterSupplier(committerSupplier)); + if (numRows >= maxRowsPerSegment) { + moveSegmentOut(ImmutableList.of(identifier)); + } + } + catch (SegmentNotWritableException e) { + throw new ISE(e, "WTF?! Segment[%s] not writable when it should have been.", identifier); + } + } + + return identifier; + } + + public int getActiveSegmentCount() + { + synchronized (activeSegments) { + return activeSegments.size(); + } + } + + public List getActiveSegments() + { + synchronized (activeSegments) { + return ImmutableList.copyOf(activeSegments.values()); + } + } + + /** + * Persist all data indexed through this driver so far. Blocks until complete. + * + * Should be called after all data has been added through {@link #add(InputRow, Supplier)}. + * + * @param committer committer representing all data that has been added so far + * + * @return commitMetadata persisted + */ + public Object persist(final Committer committer) throws InterruptedException + { + try { + log.info("Persisting data."); + final long start = System.currentTimeMillis(); + final Object commitMetadata = appenderator.persistAll(wrapCommitter(committer)).get(); + log.info("Persisted pending data in %,dms.", System.currentTimeMillis() - start); + return commitMetadata; + } + catch (InterruptedException e) { + throw e; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Publish all data indexed through this driver so far, and waits for it to be handed off. Blocks until complete. + * Retries forever on transient failures, but may exit early on permanent failures. + * + * Should be called after all data has been added and persisted through {@link #add(InputRow, Supplier)} and + * {@link #persist(Committer)}. + * + * @param publisher publisher to use for this set of segments + * @param committer committer representing all data that has been added so far + * + * @return segments and metadata published if successful, or null if segments could not be handed off due to + * transaction failure with commit metadata. + */ + public SegmentsAndMetadata finish( + final TransactionalSegmentPublisher publisher, + final Committer committer + ) throws InterruptedException + { + final SegmentsAndMetadata segmentsAndMetadata = publishAll(publisher, wrapCommitter(committer)); + + if (segmentsAndMetadata != null) { + final long giveUpAt = handoffConditionTimeout > 0 + ? System.currentTimeMillis() + handoffConditionTimeout + : 0; + + log.info("Awaiting handoff..."); + synchronized (handoffMonitor) { + while (!appenderator.getSegments().isEmpty()) { + + if (giveUpAt == 0) { + handoffMonitor.wait(); + } else { + final long remaining = giveUpAt - System.currentTimeMillis(); + if (remaining > 0) { + handoffMonitor.wait(remaining); + } else { + throw new ISE( + "Segment handoff wait timeout. Segments not yet handed off: [%s]", + Joiner.on(", ").join(appenderator.getSegments()) + ); + } + } + } + } + + return new SegmentsAndMetadata( + segmentsAndMetadata.getSegments(), + ((FiniteAppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() + ); + } else { + return null; + } + } + + /** + * Closes this driver. Does not close the underlying Appenderator; you should do that yourself. + */ + @Override + public void close() + { + handoffNotifier.close(); + } + + private SegmentIdentifier getActiveSegment(final DateTime timestamp) + { + synchronized (activeSegments) { + final Map.Entry candidateEntry = activeSegments.floorEntry(timestamp.getMillis()); + if (candidateEntry != null && candidateEntry.getValue().getInterval().contains(timestamp)) { + return candidateEntry.getValue(); + } else { + return null; + } + } + } + + /** + * Return a segment usable for "timestamp". May return null if no segment can be allocated. + * + * @param timestamp data timestamp + * + * @return identifier, or null + * + * @throws IOException if an exception occurs while allocating a segment + */ + private SegmentIdentifier getSegment(final DateTime timestamp) throws IOException + { + synchronized (activeSegments) { + final SegmentIdentifier existing = getActiveSegment(timestamp); + if (existing != null) { + return existing; + } else { + // Allocate new segment. + final SegmentIdentifier newSegment = segmentAllocator.allocate(timestamp, lastSegmentId); + + if (newSegment != null) { + final Long key = newSegment.getInterval().getStartMillis(); + final SegmentIdentifier conflicting = activeSegments.get(key); + if (conflicting != null) { + throw new ISE( + "WTF?! Allocated segment[%s] which conflicts with existing segment[%s].", + newSegment, + conflicting + ); + } + + log.info("New segment[%s].", newSegment); + activeSegments.put(key, newSegment); + lastSegmentId = newSegment.getIdentifierAsString(); + } else { + // Well, we tried. + log.warn("Cannot allocate segment for timestamp[%s]. ", timestamp); + } + + return newSegment; + } + } + } + + /** + * Move a set of identifiers out from "active", making way for newer segments. + */ + private void moveSegmentOut(final List identifiers) + { + synchronized (activeSegments) { + for (final SegmentIdentifier identifier : identifiers) { + log.info("Moving segment[%s] out of active list.", identifier); + final long key = identifier.getInterval().getStartMillis(); + if (activeSegments.remove(key) != identifier) { + throw new ISE("WTF?! Asked to remove segment[%s] that didn't exist...", identifier); + } + } + } + } + + /** + * Push and publish all segments to the metadata store. + * + * @param publisher segment publisher + * @param committer wrapped committer (from wrapCommitter) + * + * @return published segments and metadata + */ + private SegmentsAndMetadata publishAll( + final TransactionalSegmentPublisher publisher, + final Committer committer + ) throws InterruptedException + { + final List theSegments = ImmutableList.copyOf(appenderator.getSegments()); + + long nTry = 0; + while (true) { + try { + log.info("Pushing segments: [%s]", Joiner.on(", ").join(theSegments)); + final SegmentsAndMetadata segmentsAndMetadata = appenderator.push(theSegments, committer).get(); + + // Sanity check + if (!segmentsToIdentifiers(segmentsAndMetadata.getSegments()).equals(Sets.newHashSet(theSegments))) { + throw new ISE( + "WTF?! Pushed different segments than requested. Pushed[%s], requested[%s].", + Joiner.on(", ").join(identifiersToStrings(segmentsToIdentifiers(segmentsAndMetadata.getSegments()))), + Joiner.on(", ").join(identifiersToStrings(theSegments)) + ); + } + + log.info( + "Publishing segments with commitMetadata[%s]: [%s]", + segmentsAndMetadata.getCommitMetadata(), + Joiner.on(", ").join(segmentsAndMetadata.getSegments()) + ); + + final boolean published = publisher.publishSegments( + ImmutableSet.copyOf(segmentsAndMetadata.getSegments()), + ((FiniteAppenderatorDriverMetadata) segmentsAndMetadata.getCommitMetadata()).getCallerMetadata() + ); + + if (published) { + log.info("Published segments, awaiting handoff."); + } else { + log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); + if (usedSegmentChecker.findUsedSegments(segmentsToIdentifiers(segmentsAndMetadata.getSegments())) + .equals(Sets.newHashSet(segmentsAndMetadata.getSegments()))) { + log.info("Our segments really do exist, awaiting handoff."); + } else { + log.warn("Our segments don't exist, giving up."); + return null; + } + } + + for (final DataSegment dataSegment : segmentsAndMetadata.getSegments()) { + handoffNotifier.registerSegmentHandoffCallback( + new SegmentDescriptor( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().getPartitionNum() + ), + MoreExecutors.sameThreadExecutor(), + new Runnable() + { + @Override + public void run() + { + final SegmentIdentifier identifier = SegmentIdentifier.fromDataSegment(dataSegment); + log.info("Segment[%s] successfully handed off, dropping.", identifier); + final ListenableFuture dropFuture = appenderator.drop(identifier); + Futures.addCallback( + dropFuture, + new FutureCallback() + { + @Override + public void onSuccess(Object result) + { + synchronized (handoffMonitor) { + handoffMonitor.notifyAll(); + } + } + + @Override + public void onFailure(Throwable e) + { + log.warn(e, "Failed to drop segment[%s]?!"); + synchronized (handoffMonitor) { + handoffMonitor.notifyAll(); + } + } + } + ); + } + } + ); + } + + return segmentsAndMetadata; + } + catch (InterruptedException e) { + throw e; + } + catch (Exception e) { + final long sleepMillis = computeNextRetrySleep(++nTry); + log.warn(e, "Failed publishAll (try %d), retrying in %,dms.", nTry, sleepMillis); + Thread.sleep(sleepMillis); + } + } + } + + private Supplier wrapCommitterSupplier(final Supplier committerSupplier) + { + return new Supplier() + { + @Override + public Committer get() + { + return wrapCommitter(committerSupplier.get()); + } + }; + } + + private Committer wrapCommitter(final Committer committer) + { + synchronized (activeSegments) { + final FiniteAppenderatorDriverMetadata wrappedMetadata = new FiniteAppenderatorDriverMetadata( + ImmutableList.copyOf(activeSegments.values()), + lastSegmentId, + committer.getMetadata() + ); + + return new Committer() + { + @Override + public Object getMetadata() + { + return wrappedMetadata; + } + + @Override + public void run() + { + committer.run(); + } + }; + } + } + + private static long computeNextRetrySleep(final long nTry) + { + final long baseSleepMillis = 1000; + final long maxSleepMillis = 60000; + final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * new Random().nextGaussian(), 0), 2); + return (long) (Math.min(maxSleepMillis, baseSleepMillis * Math.pow(2, nTry)) * fuzzyMultiplier); + } + + private static Set segmentsToIdentifiers(final Iterable segments) + { + return FluentIterable.from(segments) + .transform( + new Function() + { + @Override + public SegmentIdentifier apply(DataSegment segment) + { + return SegmentIdentifier.fromDataSegment(segment); + } + } + ).toSet(); + } + + private static Iterable identifiersToStrings(final Iterable identifiers) + { + return FluentIterable.from(identifiers) + .transform( + new Function() + { + @Override + public String apply(SegmentIdentifier input) + { + return input.getIdentifierAsString(); + } + } + ); + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java new file mode 100644 index 000000000000..9a1f469f5da6 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverMetadata.java @@ -0,0 +1,72 @@ +/* + * 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 java.util.List; + +public class FiniteAppenderatorDriverMetadata +{ + private final List activeSegments; + private final String previousSegmentId; + private final Object callerMetadata; + + @JsonCreator + public FiniteAppenderatorDriverMetadata( + @JsonProperty("activeSegments") List activeSegments, + @JsonProperty("previousSegmentId") String previousSegmentId, + @JsonProperty("callerMetadata") Object callerMetadata + ) + { + this.activeSegments = activeSegments; + this.previousSegmentId = previousSegmentId; + this.callerMetadata = callerMetadata; + } + + @JsonProperty + public List getActiveSegments() + { + return activeSegments; + } + + @JsonProperty + public String getPreviousSegmentId() + { + return previousSegmentId; + } + + @JsonProperty + public Object getCallerMetadata() + { + return callerMetadata; + } + + @Override + public String toString() + { + return "FiniteAppenderatorDriverMetadata{" + + "activeSegments=" + activeSegments + + ", previousSegmentId='" + previousSegmentId + '\'' + + ", callerMetadata=" + callerMetadata + + '}'; + } +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java new file mode 100644 index 000000000000..65680c5539b9 --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SegmentAllocator.java @@ -0,0 +1,40 @@ +/* + * 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 org.joda.time.DateTime; + +import java.io.IOException; + +public interface SegmentAllocator +{ + /** + * Allocates a new segment for a given timestamp. + * + * @param timestamp timestamp of the event which triggered this allocation request + * @param previousSegmentId segment identifier returned on the previous call to allocate + * + * @return the pending segment identifier, or null if it was impossible to allocate a new segment + */ + SegmentIdentifier allocate( + DateTime timestamp, + String previousSegmentId + ) throws IOException; +} 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/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/io/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java new file mode 100644 index 000000000000..de74f58966ac --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -0,0 +1,40 @@ +/* + * 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 io.druid.timeline.DataSegment; + +import java.io.IOException; +import java.util.Set; + +public interface TransactionalSegmentPublisher +{ + /** + * Publish segments, along with some commit metadata, in a single transaction. + * + * @return true if segments were published, false if they were not published due to txn failure with the metadata + * + * @throws IOException if there was an I/O error when publishing + */ + boolean publishSegments( + Set segments, + Object commitMetadata + ) throws IOException; +} diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/UsedSegmentChecker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/UsedSegmentChecker.java new file mode 100644 index 000000000000..aeb0e99f13cb --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/UsedSegmentChecker.java @@ -0,0 +1,37 @@ +/* + * 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 io.druid.timeline.DataSegment; + +import java.io.IOException; +import java.util.Set; + +public interface UsedSegmentChecker +{ + /** + * For any identifiers that exist and are actually used, returns the corresponding DataSegment objects. + * + * @param identifiers identifiers to search for + * + * @return used DataSegments + */ + Set findUsedSegments(Set identifiers) throws IOException; +} diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 238ec238edc1..e6e7e4483c13 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -255,7 +255,14 @@ private Sink getSink(long timestamp) segmentGranularity.increment(new DateTime(truncatedTime)) ); - retVal = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval)); + retVal = new Sink( + sinkInterval, + schema, + config.getShardSpec(), + versioningPolicy.getVersion(sinkInterval), + config.getMaxRowsInMemory(), + config.isReportParseExceptions() + ); addSink(retVal); } @@ -822,7 +829,15 @@ public int compare(File o1, File o2) ); continue; } - final Sink currSink = new Sink(sinkInterval, schema, config, versioningPolicy.getVersion(sinkInterval), hydrants); + final Sink currSink = new Sink( + sinkInterval, + schema, + config.getShardSpec(), + versioningPolicy.getVersion(sinkInterval), + config.getMaxRowsInMemory(), + config.isReportParseExceptions(), + hydrants + ); addSink(currSink); } return metadata; 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 c6e4bb712650..91326192fdd9 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 @@ -30,50 +30,56 @@ import io.druid.data.input.InputRow; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.QueryableIndex; -import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.incremental.OnheapIncrementalIndex; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireHydrant; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Arrays; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; -/** - */ public class Sink implements Iterable { + private static final int ADD_FAILED = -1; private final Object hydrantLock = new Object(); private final Interval interval; private final DataSchema schema; - private final RealtimeTuningConfig config; + private final ShardSpec shardSpec; private final String version; + private final int maxRowsInMemory; + private final boolean reportParseExceptions; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList(); private final LinkedHashSet dimOrder = Sets.newLinkedHashSet(); + private final AtomicInteger numRowsExcludingCurrIndex = new AtomicInteger(); private volatile FireHydrant currHydrant; + private volatile boolean writable = true; public Sink( Interval interval, DataSchema schema, - RealtimeTuningConfig config, - String version + ShardSpec shardSpec, + String version, + int maxRowsInMemory, + boolean reportParseExceptions ) { this.schema = schema; - this.config = config; + this.shardSpec = shardSpec; this.interval = interval; this.version = version; + this.maxRowsInMemory = maxRowsInMemory; + this.reportParseExceptions = reportParseExceptions; makeNewCurrIndex(interval.getStartMillis(), schema); } @@ -81,15 +87,19 @@ public Sink( public Sink( Interval interval, DataSchema schema, - RealtimeTuningConfig config, + ShardSpec shardSpec, String version, + int maxRowsInMemory, + boolean reportParseExceptions, List hydrants ) { this.schema = schema; - this.config = config; + this.shardSpec = shardSpec; this.interval = interval; this.version = version; + this.maxRowsInMemory = maxRowsInMemory; + this.reportParseExceptions = reportParseExceptions; int maxCount = -1; for (int i = 0; i < hydrants.size(); ++i) { @@ -98,6 +108,7 @@ public Sink( throw new ISE("hydrant[%s] not the right count[%s]", hydrant, i); } maxCount = hydrant.getCount(); + numRowsExcludingCurrIndex.addAndGet(hydrant.getSegment().asQueryableIndex().getNumRows()); } this.hydrants.addAll(hydrants); @@ -126,9 +137,13 @@ public int add(InputRow row) throws IndexSizeExceededException } synchronized (hydrantLock) { + if (!writable) { + return ADD_FAILED; + } + IncrementalIndex index = currHydrant.getIndex(); if (index == null) { - return -1; // the hydrant was swapped without being replaced + return ADD_FAILED; // the hydrant was swapped without being replaced } return index.add(row); } @@ -136,8 +151,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(); } } @@ -148,6 +163,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. * @@ -161,7 +181,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; } } @@ -183,12 +215,19 @@ public String apply(@Nullable AggregatorFactory input) } } ), - config.getShardSpec(), + shardSpec, null, 0 ); } + public int getNumRows() + { + synchronized (hydrantLock) { + return numRowsExcludingCurrIndex.get() + currHydrant.getIndex().size(); + } + } + private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() @@ -197,35 +236,40 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) .withDimensionsSpec(schema.getParser()) .withMetrics(schema.getAggregators()) .build(); - final IncrementalIndex newIndex = new OnheapIncrementalIndex( - indexSchema, - config.isReportParseExceptions(), - config.getMaxRowsInMemory() - ); + final IncrementalIndex newIndex = new OnheapIncrementalIndex(indexSchema, reportParseExceptions, maxRowsInMemory); 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 (!indexSchema.getDimensionsSpec().hasCustomDimensions()) { - if (lastHydrant.hasSwapped()) { - QueryableIndex oldIndex = lastHydrant.getSegment().asQueryableIndex(); - for (String dim : oldIndex.getAvailableDimensions()) { - dimOrder.add(dim); + if (writable) { + old = currHydrant; + int newCount = 0; + int numHydrants = hydrants.size(); + if (numHydrants > 0) { + FireHydrant lastHydrant = hydrants.get(numHydrants - 1); + newCount = lastHydrant.getCount() + 1; + if (!indexSchema.getDimensionsSpec().hasCustomDimensions()) { + if (lastHydrant.hasSwapped()) { + QueryableIndex oldIndex = lastHydrant.getSegment().asQueryableIndex(); + for (String dim : oldIndex.getAvailableDimensions()) { + dimOrder.add(dim); + } + } else { + IncrementalIndex oldIndex = lastHydrant.getIndex(); + dimOrder.addAll(oldIndex.getDimensionOrder()); } - } else { - IncrementalIndex oldIndex = lastHydrant.getIndex(); - dimOrder.addAll(oldIndex.getDimensionOrder()); + newIndex.loadDimensionIterable(dimOrder); } - newIndex.loadDimensionIterable(dimOrder); } + currHydrant = new FireHydrant(newIndex, newCount, getSegment().getIdentifier()); + if (old != null) { + numRowsExcludingCurrIndex.addAndGet(old.getIndex().size()); + } + 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; @@ -239,7 +283,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/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index e74dbbe9cbb7..b8edd87bc3df 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -203,7 +203,14 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException null, null ); - plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString())); + plumber = new TestPlumber(new Sink( + new Interval("0/P5000Y"), + schema, + tuningConfig.getShardSpec(), + new DateTime().toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + )); realtimeManager = new RealtimeManager( Arrays.asList( @@ -215,7 +222,14 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException ), null ); - plumber2 = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema2, tuningConfig, new DateTime().toString())); + plumber2 = new TestPlumber(new Sink( + new Interval("0/P5000Y"), + schema2, + tuningConfig.getShardSpec(), + new DateTime().toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + )); realtimeManager2 = new RealtimeManager( Arrays.asList( 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..3b910bd8cbdf --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -0,0 +1,456 @@ +/* + * 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); + + // push all + final SegmentsAndMetadata segmentsAndMetadata = appenderator.push( + appenderator.getSegments(), + 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..88dc4b0eb7db --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -0,0 +1,286 @@ +/* + * 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.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; +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.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 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, + null, + 0, + 0, + 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); + + 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.createRealtime( + 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 + { + + } + + @Override + public boolean isAnnounced(DataSegment segment) + { + return false; + } + }, + emitter, + queryExecutor, + MapCache.create(2048), + new CacheConfig() + ); + } + + 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)); + } +} diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java new file mode 100644 index 000000000000..2ed509a5d697 --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/FiniteAppenderatorDriverTest.java @@ -0,0 +1,296 @@ +/* + * 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.api.client.util.Maps; +import com.google.api.client.util.Sets; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; +import com.metamx.common.Granularity; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.SegmentDescriptor; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.PartitionChunk; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class FiniteAppenderatorDriverTest +{ + private static final String DATA_SOURCE = "foo"; + private static final String VERSION = "abc123"; + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private static final int MAX_ROWS_IN_MEMORY = 100; + private static final int MAX_ROWS_PER_SEGMENT = 3; + private static final long HANDOFF_CONDITION_TIMEOUT = 0; + + private static final List ROWS = Arrays.asList( + new MapBasedInputRow( + new DateTime("2000"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo", "met1", "1") + ), + new MapBasedInputRow( + new DateTime("2000T01"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo", "met1", 2.0) + ), + new MapBasedInputRow( + new DateTime("2000T01"), + ImmutableList.of("dim2"), + ImmutableMap.of("dim2", "bar", "met1", 2.0) + ) + ); + + AppenderatorTester appenderatorTester; + FiniteAppenderatorDriver driver; + + @Before + public void setUp() + { + appenderatorTester = new AppenderatorTester(MAX_ROWS_IN_MEMORY); + driver = new FiniteAppenderatorDriver( + appenderatorTester.getAppenderator(), + new TestSegmentAllocator(DATA_SOURCE, Granularity.HOUR), + new TestSegmentHandoffNotifierFactory(), + new TestUsedSegmentChecker(), + OBJECT_MAPPER, + MAX_ROWS_PER_SEGMENT, + HANDOFF_CONDITION_TIMEOUT + ); + } + + @After + public void tearDown() throws Exception + { + driver.clear(); + driver.close(); + } + + @Test + public void testSimple() throws Exception + { + final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); + + Assert.assertNull(driver.startJob()); + + for (int i = 0; i < ROWS.size(); i++) { + committerSupplier.setMetadata(i + 1); + Assert.assertNotNull(driver.add(ROWS.get(i), committerSupplier)); + } + + final SegmentsAndMetadata segmentsAndMetadata = driver.finish( + makeOkPublisher(), + committerSupplier.get() + ); + + Assert.assertEquals( + ImmutableSet.of( + new SegmentIdentifier(DATA_SOURCE, new Interval("2000/PT1H"), VERSION, new NumberedShardSpec(0, 0)), + new SegmentIdentifier(DATA_SOURCE, new Interval("2000T01/PT1H"), VERSION, new NumberedShardSpec(0, 0)) + ), + asIdentifiers(segmentsAndMetadata.getSegments()) + ); + + Assert.assertEquals(3, segmentsAndMetadata.getCommitMetadata()); + } + + private Set asIdentifiers(Iterable segments) + { + return ImmutableSet.copyOf( + Iterables.transform( + segments, + new Function() + { + @Override + public SegmentIdentifier apply(DataSegment input) + { + return SegmentIdentifier.fromDataSegment(input); + } + } + ) + ); + } + + private TransactionalSegmentPublisher makeOkPublisher() + { + return new TransactionalSegmentPublisher() + { + @Override + public boolean publishSegments(Set segments, Object commitMetadata) throws IOException + { + return true; + } + }; + } + + private class TestCommitterSupplier implements Supplier + { + private final AtomicReference metadata = new AtomicReference<>(); + + public void setMetadata(T newMetadata) + { + metadata.set(newMetadata); + } + + @Override + public Committer get() + { + final T currentMetadata = metadata.get(); + return new Committer() + { + @Override + public Object getMetadata() + { + return currentMetadata; + } + + @Override + public void run() + { + // Do nothing + } + }; + } + } + + private class TestSegmentAllocator implements SegmentAllocator + { + private final String dataSource; + private final Granularity granularity; + private final Map counters = Maps.newHashMap(); + + public TestSegmentAllocator(String dataSource, Granularity granularity) + { + this.dataSource = dataSource; + this.granularity = granularity; + } + + @Override + public SegmentIdentifier allocate( + final DateTime timestamp, + final String previousSegmentId + ) throws IOException + { + synchronized (counters) { + final long timestampTruncated = granularity.truncate(timestamp).getMillis(); + if (!counters.containsKey(timestampTruncated)) { + counters.put(timestampTruncated, new AtomicInteger()); + } + final int partitionNum = counters.get(timestampTruncated).getAndIncrement(); + return new SegmentIdentifier( + dataSource, + granularity.bucket(new DateTime(timestampTruncated)), + VERSION, + new NumberedShardSpec(partitionNum, 0) + ); + } + } + } + + private class TestSegmentHandoffNotifierFactory implements SegmentHandoffNotifierFactory + { + @Override + public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) + { + return new SegmentHandoffNotifier() + { + @Override + public boolean registerSegmentHandoffCallback( + final SegmentDescriptor descriptor, + final Executor exec, + final Runnable handOffRunnable + ) + { + // Immediate handoff + exec.execute(handOffRunnable); + return true; + } + + @Override + public void start() + { + // Do nothing + } + + @Override + public void close() + { + // Do nothing + } + }; + } + } + + private class TestUsedSegmentChecker implements UsedSegmentChecker + { + @Override + public Set findUsedSegments(Set identifiers) throws IOException + { + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { + timeline.add( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().createChunk(dataSegment) + ); + } + + final Set retVal = Sets.newHashSet(); + for (SegmentIdentifier identifier : identifiers) { + for (TimelineObjectHolder holder : timeline.lookup(identifier.getInterval())) { + for (PartitionChunk chunk : holder.getObject()) { + if (identifiers.contains(SegmentIdentifier.fromDataSegment(chunk.getObject()))) { + retVal.add(chunk.getObject()); + } + } + } + } + + return retVal; + } + } +} 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 fcc704fff8ae..55eb86dbfc2d 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 @@ -257,8 +257,10 @@ private void testPersist(final Object commitMetadata) throws Exception new Sink( new Interval(0, TimeUnit.HOURS.toMillis(1)), schema, - tuningConfig, - new DateTime("2014-12-01T12:34:56.789").toString() + tuningConfig.getShardSpec(), + new DateTime("2014-12-01T12:34:56.789").toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() ) ); Assert.assertNull(plumber.startJob()); @@ -302,8 +304,10 @@ public void testPersistFails() throws Exception new Sink( new Interval(0, TimeUnit.HOURS.toMillis(1)), schema, - tuningConfig, - new DateTime("2014-12-01T12:34:56.789").toString() + tuningConfig.getShardSpec(), + new DateTime("2014-12-01T12:34:56.789").toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() ) ); plumber.startJob(); @@ -357,8 +361,10 @@ private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Ex new Sink( testInterval, schema2, - tuningConfig, - new DateTime("2014-12-01T12:34:56.789").toString() + tuningConfig.getShardSpec(), + new DateTime("2014-12-01T12:34:56.789").toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() ) ); Assert.assertNull(plumber2.startJob()); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 5225752aeaf8..612853e5594e 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -73,7 +73,14 @@ public void testSwap() throws Exception null, null ); - final Sink sink = new Sink(interval, schema, tuningConfig, version); + final Sink sink = new Sink( + interval, + schema, + tuningConfig.getShardSpec(), + version, + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + ); sink.add( new InputRow() From 187569e702437e8f37880bd78ef76001f70b2cb7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 1 Mar 2016 16:51:35 -0800 Subject: [PATCH 5/6] DataSource metadata. Geared towards supporting transactional inserts of new segments. This involves an interface "DataSourceMetadata" that allows combining of partially specified metadata (useful for partitioned ingestion). DataSource metadata is stored in a new "dataSource" table. --- .../metadata/MetadataStorageConnector.java | 3 +- .../metadata/MetadataStorageTablesConfig.java | 12 +- .../postgresql/PostgreSQLConnectorTest.java | 15 +- .../MetadataStorageUpdaterJobSpec.java | 1 + .../updater/HadoopConverterJobTest.java | 1 + .../io/druid/indexing/common/TaskToolbox.java | 3 +- .../common/actions/SegmentInsertAction.java | 60 +++- .../common/task/HadoopConverterTask.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../indexing/common/task/MergeTaskBase.java | 2 +- .../common/task/RealtimeIndexTask.java | 2 +- .../io/druid/indexing/common/TestTask.java | 29 -- .../actions/SegmentInsertActionTest.java | 179 ++++++++++ .../common/actions/TaskActionTestKit.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 18 +- ...TestIndexerMetadataStorageCoordinator.java | 19 ++ .../indexing/overlord/DataSourceMetadata.java | 74 +++++ .../IndexerMetadataStorageCoordinator.java | 39 ++- .../indexing/overlord/ObjectMetadata.java | 87 +++++ .../overlord/SegmentPublishResult.java | 99 ++++++ .../IndexerSQLMetadataStorageCoordinator.java | 308 +++++++++++++++--- .../druid/metadata/SQLMetadataConnector.java | 74 ++++- ...exerSQLMetadataStorageCoordinatorTest.java | 109 ++++++- .../main/java/io/druid/cli/CreateTables.java | 1 + 25 files changed, 1000 insertions(+), 142 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 create mode 100644 server/src/main/java/io/druid/indexing/overlord/DataSourceMetadata.java create mode 100644 server/src/main/java/io/druid/indexing/overlord/ObjectMetadata.java create mode 100644 server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.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-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index 21c7201748ef..2fda23303814 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/io/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/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 3c7e92df8973..45549c28f892 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 @@ -209,6 +209,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/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 36767cc24d24..d9a690eb65ba 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -206,7 +206,7 @@ public Map fetchSegments(List segments) return retVal; } - public void pushSegments(Iterable segments) throws IOException + public void publishSegments(Iterable segments) throws IOException { // Request segment pushes for each set final Multimap segmentMultimap = Multimaps.index( @@ -223,7 +223,6 @@ public Interval apply(DataSegment segment) for (final Collection segmentCollection : segmentMultimap.asMap().values()) { getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(segmentCollection))); } - } public File getTaskWorkDir() 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..3240e5832b32 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,12 +20,13 @@ 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; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.DataSourceMetadata; +import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.query.DruidMetrics; import io.druid.timeline.DataSegment; @@ -40,17 +41,29 @@ * that the task cannot actually complete. Callers should avoid this by avoiding inserting too many segments in the * same action. */ -public class SegmentInsertAction implements TaskAction> +public class SegmentInsertAction implements TaskAction { - @JsonIgnore private final Set segments; + private final DataSourceMetadata startMetadata; + private final DataSourceMetadata endMetadata; + + public SegmentInsertAction( + Set segments + ) + { + this(segments, null, null); + } @JsonCreator public SegmentInsertAction( - @JsonProperty("segments") Set segments + @JsonProperty("segments") Set segments, + @JsonProperty("startMetadata") DataSourceMetadata startMetadata, + @JsonProperty("endMetadata") DataSourceMetadata endMetadata ) { this.segments = ImmutableSet.copyOf(segments); + this.startMetadata = startMetadata; + this.endMetadata = endMetadata; } @JsonProperty @@ -59,26 +72,53 @@ public Set getSegments() return segments; } - public TypeReference> getReturnTypeReference() + @JsonProperty + public DataSourceMetadata getStartMetadata() { - return new TypeReference>() + return startMetadata; + } + + @JsonProperty + public DataSourceMetadata getEndMetadata() + { + return endMetadata; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() { }; } + /** + * Behaves similarly to + * {@link io.druid.indexing.overlord.IndexerMetadataStorageCoordinator#announceHistoricalSegments(Set, DataSourceMetadata, DataSourceMetadata)}, + * including the possibility of returning null in case of metadata transaction failure. + */ @Override - public Set perform(Task task, TaskActionToolbox toolbox) throws IOException + public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throws IOException { toolbox.verifyTaskLocks(task, segments); - final Set retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments(segments); + final SegmentPublishResult retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + segments, + startMetadata, + endMetadata + ); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) .setDimension(DruidMetrics.TASK_TYPE, task.getType()); - for (DataSegment segment : segments) { + if (retVal.isSuccess()) { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + } else { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + } + + for (DataSegment segment : retVal.getSegments()) { metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); } @@ -97,6 +137,8 @@ public String toString() { return "SegmentInsertAction{" + "segments=" + segments + + ", startMetadata=" + startMetadata + + ", endMetadata=" + endMetadata + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java index a3f76dc43184..08d45cbc8d70 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java @@ -249,7 +249,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception } ); log.debug("Found new segments %s", Arrays.toString(finishedSegments.toArray())); - toolbox.pushSegments(finishedSegments); + toolbox.publishSegments(finishedSegments); return success(); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 83ca7f614ef4..28f177f602c8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -222,7 +222,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception } ); - toolbox.pushSegments(publishedSegments); + toolbox.publishSegments(publishedSegments); return TaskStatus.success(getId()); } else { return TaskStatus.failure(getId()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 3ba44f90596a..eb56b37322ce 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -228,7 +228,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception segments.add(segment); } } - toolbox.pushSegments(segments); + toolbox.publishSegments(segments); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 9a31916d6c37..dee4ae29d7af 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -171,7 +171,7 @@ public String apply(DataSegment input) emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - toolbox.pushSegments(ImmutableList.of(uploadedSegment)); + toolbox.publishSegments(ImmutableList.of(uploadedSegment)); return TaskStatus.success(getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 45deeffed410..4b2a93ef031d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -514,7 +514,7 @@ public TaskActionSegmentPublisher(Task task, TaskToolbox taskToolbox) @Override public void publishSegment(DataSegment segment) throws IOException { - taskToolbox.pushSegments(ImmutableList.of(segment)); + taskToolbox.publishSegments(ImmutableList.of(segment)); } } } 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..4338829d352a --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -0,0 +1,179 @@ +/* + * 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.indexing.overlord.ObjectMetadata; +import io.druid.indexing.overlord.SegmentPublishResult; +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)); + + SegmentPublishResult result1 = new SegmentInsertAction( + ImmutableSet.of(SEGMENT1), + new ObjectMetadata(null), + new ObjectMetadata(ImmutableList.of(1)) + ).perform( + task, + actionTestKit.getTaskActionToolbox() + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(SEGMENT1), true), result1); + + SegmentPublishResult result2 = new SegmentInsertAction( + ImmutableSet.of(SEGMENT2), + new ObjectMetadata(ImmutableList.of(1)), + new ObjectMetadata(ImmutableList.of(2)) + ).perform( + task, + actionTestKit.getTaskActionToolbox() + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(SEGMENT2), true), result2); + + Assert.assertEquals( + ImmutableSet.of(SEGMENT1, SEGMENT2), + ImmutableSet.copyOf( + actionTestKit.getMetadataStorageCoordinator() + .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL) + ) + ); + + Assert.assertEquals( + new ObjectMetadata(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")); + SegmentPublishResult result = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(SEGMENT3), true), result); + } + + @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)); + + SegmentPublishResult result = new SegmentInsertAction( + ImmutableSet.of(SEGMENT1), + new ObjectMetadata(ImmutableList.of(1)), + new ObjectMetadata(ImmutableList.of(2)) + ).perform( + task, + actionTestKit.getTaskActionToolbox() + ); + + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(), false), result); + } +} 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 d4d335800a58..abc32ebec62a 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 @@ -39,9 +39,7 @@ import com.metamx.common.Pair; 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.cache.MapCache; @@ -102,6 +100,7 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -218,20 +217,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) 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..3c2bf2917eef 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 @@ -23,7 +23,9 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -43,6 +45,12 @@ public TestIndexerMetadataStorageCoordinator() unusedSegments = Lists.newArrayList(); } + @Override + public DataSourceMetadata getDataSourceMetadata(String dataSource) + { + throw new UnsupportedOperationException(); + } + @Override public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException { @@ -77,6 +85,17 @@ public Set announceHistoricalSegments(Set segments) return ImmutableSet.copyOf(added); } + @Override + public SegmentPublishResult announceHistoricalSegments( + Set segments, + DataSourceMetadata oldCommitMetadata, + DataSourceMetadata newCommitMetadata + ) throws IOException + { + // Don't actually compare metadata, just do it! + return new SegmentPublishResult(announceHistoricalSegments(segments), true); + } + @Override public SegmentIdentifier allocatePendingSegment( String dataSource, diff --git a/server/src/main/java/io/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/io/druid/indexing/overlord/DataSourceMetadata.java new file mode 100644 index 000000000000..29551c8734c9 --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/DataSourceMetadata.java @@ -0,0 +1,74 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.overlord; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import java.util.Set; + +/** + * Commit metadata for a dataSource. Used by + * {@link IndexerMetadataStorageCoordinator#announceHistoricalSegments(Set, DataSourceMetadata, DataSourceMetadata)} + * to provide metadata transactions for segment inserts. + * + * Two metadata instances can be added together, and any conflicts are resolved in favor of the right-hand side. + * This means metadata can be partitioned. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "object", value = ObjectMetadata.class) +}) +public interface DataSourceMetadata +{ + /** + * Returns true if this instance should be considered a valid starting point for a new dataSource that has + * no existing metadata. + */ + boolean isValidStart(); + + /** + * Returns true if any information present in this instance matches analogous information from "other" and + * so they are conflict-free. In other words, "one.plus(two)" and "two.plus(one)" should return equal + * instances if "one" matches "two". + * + * One simple way to implement this is to make it the same as "equals", although that doesn't allow for + * partitioned metadata. + * + * Behavior is undefined if you pass in an instance of a different class from this one. + * + * @param other another instance + * + * @return true or false + */ + boolean matches(DataSourceMetadata other); + + /** + * Returns a copy of this instance with "other" merged in. Any conflicts should be resolved in favor of + * information from "other". + * + * Behavior is undefined if you pass in an instance of a different class from this one. + * + * @param other another instance + * + * @return merged copy + */ + DataSourceMetadata plus(DataSourceMetadata other); +} 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..d20e5cdc01ae 100644 --- a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -41,14 +41,14 @@ public interface IndexerMetadataStorageCoordinator * * @throws IOException */ - public List getUsedSegmentsForInterval(final String dataSource, final Interval interval) + List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException; /** * Get all segments which may include any data in the interval and are flagged as used. * * @param dataSource The datasource to query - * @param intervals The intervals for which all applicable and used datasources are requested. + * @param intervals The intervals for which all applicable and used datasources are requested. * * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the requested interval. * @@ -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,36 @@ 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 startMetadata and endMetadata are 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 startMetadata dataSource metadata pre-insert must match this startMetadata according to + * {@link DataSourceMetadata#matches(DataSourceMetadata)}. If null, this insert will + * not involve a metadata transaction + * @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with + * {@link DataSourceMetadata#plus(DataSourceMetadata)}. If null, this insert will not + * involve a metadata transaction + * + * @return segment publish result indicating transaction success or failure, and set of segments actually published + * + * @throws IllegalArgumentException if startMetadata and endMetadata are not either both null or both non-null + */ + SegmentPublishResult announceHistoricalSegments( + Set segments, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) throws IOException; + + DataSourceMetadata 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 +132,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/indexing/overlord/ObjectMetadata.java b/server/src/main/java/io/druid/indexing/overlord/ObjectMetadata.java new file mode 100644 index 000000000000..b3989f770731 --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/ObjectMetadata.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.indexing.overlord; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public final class ObjectMetadata implements DataSourceMetadata +{ + private final Object theObject; + + @JsonCreator + public ObjectMetadata( + @JsonProperty("object") Object theObject + ) + { + this.theObject = theObject; + } + + @JsonProperty("object") + public Object getObject() + { + return theObject; + } + + @Override + public boolean isValidStart() + { + return theObject == null; + } + + @Override + public boolean matches(DataSourceMetadata other) + { + return equals(other); + } + + @Override + public DataSourceMetadata plus(DataSourceMetadata other) + { + return other; + } + + @Override + public boolean equals(Object o) + { + if (o instanceof ObjectMetadata) { + final Object other = ((ObjectMetadata) o).getObject(); + return (theObject == null && other == null) || (theObject != null && theObject.equals(other)); + } else { + return false; + } + } + + @Override + public int hashCode() + { + return Objects.hash(theObject); + } + + @Override + public String toString() + { + return "ObjectMetadata{" + + "theObject=" + theObject + + '}'; + } +} diff --git a/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java b/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java new file mode 100644 index 000000000000..02545222c9d8 --- /dev/null +++ b/server/src/main/java/io/druid/indexing/overlord/SegmentPublishResult.java @@ -0,0 +1,99 @@ +/* + * 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.overlord; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.druid.timeline.DataSegment; + +import java.util.Objects; +import java.util.Set; + +/** + * Result of an operation that attempts to publish segments. Indicates the set of segments actually published + * and whether or not the transaction was a success. + * + * If "success" is false then the segments set will be empty. + * + * It's possible for the segments set to be empty even if "success" is true, since the segments set only + * includes segments actually published as part of the transaction. The requested segments could have been + * published by a different transaction (e.g. in the case of replica sets) and this one would still succeed. + */ +public class SegmentPublishResult +{ + private final Set segments; + private final boolean success; + + @JsonCreator + public SegmentPublishResult( + @JsonProperty("segments") Set segments, + @JsonProperty("success") boolean success + ) + { + this.segments = Preconditions.checkNotNull(segments, "segments"); + this.success = success; + + if (!success) { + Preconditions.checkArgument(segments.isEmpty(), "segments must be empty for unsuccessful publishes"); + } + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + @JsonProperty + public boolean isSuccess() + { + return success; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SegmentPublishResult that = (SegmentPublishResult) o; + return success == that.success && + Objects.equals(segments, that.segments); + } + + @Override + public int hashCode() + { + return Objects.hash(segments, success); + } + + @Override + public String toString() + { + return "SegmentPublishResult{" + + "segments=" + segments + + ", success=" + success + + '}'; + } +} diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 6128b2db7052..db64cb6fa6df 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -33,10 +33,13 @@ import com.google.common.io.BaseEncoding; import com.google.inject.Inject; import com.metamx.common.IAE; +import com.metamx.common.ISE; import com.metamx.common.StringUtils; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.logger.Logger; +import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; @@ -55,6 +58,7 @@ import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.TransactionCallback; import org.skife.jdbi.v2.TransactionStatus; +import org.skife.jdbi.v2.exceptions.CallbackFailedException; import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.util.ByteArrayMapper; import org.skife.jdbi.v2.util.StringMapper; @@ -65,14 +69,14 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; /** */ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator { private static final Logger log = new Logger(IndexerSQLMetadataStorageCoordinator.class); - - private static int ALLOCATE_SEGMENT_QUIET_TRIES = 3; + private static final int ALLOCATE_SEGMENT_QUIET_TRIES = 3; private final ObjectMapper jsonMapper; private final MetadataStorageTablesConfig dbTables; @@ -93,6 +97,7 @@ public IndexerSQLMetadataStorageCoordinator( @LifecycleStart public void start() { + connector.createDataSourceTable(); connector.createPendingSegmentsTable(); connector.createSegmentTable(); } @@ -264,24 +269,90 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi */ public Set announceHistoricalSegments(final Set segments) throws IOException { - return connector.getDBI().inTransaction( - new TransactionCallback>() - { - @Override - public Set inTransaction(Handle handle, TransactionStatus transactionStatus) throws IOException + final SegmentPublishResult result = announceHistoricalSegments(segments, null, null); + + // Metadata transaction cannot fail because we are not trying to do one. + if (!result.isSuccess()) { + throw new ISE("WTF?! announceHistoricalSegments failed with null metadata, should not happen."); + } + + return result.getSegments(); + } + + /** + * {@inheritDoc} + */ + @Override + public SegmentPublishResult announceHistoricalSegments( + final Set segments, + final DataSourceMetadata startMetadata, + final DataSourceMetadata endMetadata + ) throws IOException + { + 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"); + } + } + + if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { + throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); + } + + final AtomicBoolean txnFailure = new AtomicBoolean(false); + + try { + return connector.retryTransaction( + new TransactionCallback() { - final Set inserted = Sets.newHashSet(); + @Override + public SegmentPublishResult inTransaction( + final Handle handle, + final TransactionStatus transactionStatus + ) throws Exception + { + final Set inserted = Sets.newHashSet(); + + if (startMetadata != null) { + final boolean success = updateDataSourceMetadataWithHandle( + handle, + dataSource, + startMetadata, + endMetadata + ); - for (final DataSegment segment : segments) { - if (announceHistoricalSegment(handle, segment)) { - inserted.add(segment); + if (!success) { + transactionStatus.setRollbackOnly(); + txnFailure.set(true); + throw new RuntimeException("Aborting transaction!"); + } } - } - return ImmutableSet.copyOf(inserted); - } - } - ); + for (final DataSegment segment : segments) { + if (announceHistoricalSegment(handle, segment)) { + inserted.add(segment); + } + } + + return new SegmentPublishResult(ImmutableSet.copyOf(inserted), true); + } + }, + 3, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + } + catch (CallbackFailedException e) { + if (txnFailure.get()) { + return new SegmentPublishResult(ImmutableSet.of(), false); + } else { + throw e; + } + } } @Override @@ -499,10 +570,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 { @@ -512,38 +583,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; } @@ -564,6 +628,150 @@ private boolean segmentExists(final Handle handle, final DataSegment segment) .isEmpty(); } + /** + * Read dataSource metadata. Returns null if there is no metadata. + */ + public DataSourceMetadata getDataSourceMetadata(final String dataSource) + { + final byte[] bytes = connector.lookup( + dbTables.getDataSourceTable(), + "dataSource", + "commit_metadata_payload", + dataSource + ); + + if (bytes == null) { + return null; + } + + try { + return jsonMapper.readValue(bytes, DataSourceMetadata.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. This will only modify dataSource metadata if it equals + * oldCommitMetadata when this function is called (based on T.equals). This method is idempotent in that if + * the metadata already equals newCommitMetadata, it will return true. + * + * @param handle database handle + * @param dataSource druid dataSource + * @param startMetadata dataSource metadata pre-insert must match this startMetadata according to + * {@link DataSourceMetadata#matches(DataSourceMetadata)} + * @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with + * {@link DataSourceMetadata#plus(DataSourceMetadata)} + * + * @return true if dataSource metadata was updated from matching startMetadata to matching endMetadata + */ + private boolean updateDataSourceMetadataWithHandle( + final Handle handle, + final String dataSource, + final DataSourceMetadata startMetadata, + final DataSourceMetadata endMetadata + ) throws IOException + { + Preconditions.checkNotNull(dataSource, "dataSource"); + Preconditions.checkNotNull(startMetadata, "startMetadata"); + Preconditions.checkNotNull(endMetadata, "endMetadata"); + + final byte[] oldCommitMetadataBytesFromDb = getDataSourceMetadataWithHandleAsBytes(handle, dataSource); + final String oldCommitMetadataSha1FromDb; + final DataSourceMetadata oldCommitMetadataFromDb; + + if (oldCommitMetadataBytesFromDb == null) { + oldCommitMetadataSha1FromDb = null; + oldCommitMetadataFromDb = null; + } else { + oldCommitMetadataSha1FromDb = BaseEncoding.base16().encode( + Hashing.sha1().hashBytes(oldCommitMetadataBytesFromDb).asBytes() + ); + oldCommitMetadataFromDb = jsonMapper.readValue(oldCommitMetadataBytesFromDb, DataSourceMetadata.class); + } + + final boolean startMetadataMatchesExisting = oldCommitMetadataFromDb == null + ? startMetadata.isValidStart() + : startMetadata.matches(oldCommitMetadataFromDb); + + if (!startMetadataMatchesExisting) { + // Not in the desired start state. + log.info("Not updating metadata, existing state is not the expected start state."); + return false; + } + + final DataSourceMetadata newCommitMetadata = oldCommitMetadataFromDb == null + ? endMetadata + : oldCommitMetadataFromDb.plus(endMetadata); + final byte[] newCommitMetadataBytes = jsonMapper.writeValueAsBytes(newCommitMetadata); + final String newCommitMetadataSha1 = BaseEncoding.base16().encode( + Hashing.sha1().hashBytes(newCommitMetadataBytes).asBytes() + ); + + final boolean retVal; + if (oldCommitMetadataBytesFromDb == null) { + // 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(); + + retVal = numRows > 0; + } else { + // Expecting a particular old metadata; 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(); + + retVal = numRows > 0; + } + + if (retVal) { + log.info("Updated metadata from[%s] to[%s].", oldCommitMetadataFromDb, newCommitMetadata); + } else { + log.info("Not updating metadata, compare-and-swap failure."); + } + + return retVal; + } + 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 b0c7b1f690cd..5a0ade98ae91 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataConnector.java @@ -216,6 +216,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( @@ -390,6 +409,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() { @@ -442,35 +468,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() diff --git a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 4da60a093cbd..b0078f4268fc 100644 --- a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.druid.indexing.overlord.ObjectMetadata; +import io.druid.indexing.overlord.SegmentPublishResult; import io.druid.jackson.DefaultObjectMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; @@ -32,6 +34,7 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.tweak.HandleCallback; @@ -42,9 +45,10 @@ public class IndexerSQLMetadataStorageCoordinatorTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + 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 +60,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest ); private final DataSegment defaultSegment2 = new DataSegment( - "dataSource", + "fooDataSource", Interval.parse("2015-01-01T00Z/2015-01-02T00Z"), "version", ImmutableMap.of(), @@ -68,7 +72,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest ); private final DataSegment defaultSegment3 = new DataSegment( - "dataSource", + "fooDataSource", Interval.parse("2015-01-03T00Z/2015-01-04T00Z"), "version", ImmutableMap.of(), @@ -88,6 +92,7 @@ public void setUp() { derbyConnector = derbyConnectorRule.getConnector(); mapper.registerSubtypes(LinearShardSpec.class); + derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); coordinator = new IndexerSQLMetadataStorageCoordinator( @@ -135,6 +140,99 @@ public void testSimpleAnnounce() throws IOException ); } + @Test + public void testTransactionalAnnounceSuccess() throws IOException + { + // Insert first segment. + final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + new ObjectMetadata(null), + new ObjectMetadata(ImmutableMap.of("foo", "bar")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(defaultSegment), true), result1); + + Assert.assertArrayEquals( + mapper.writeValueAsString(defaultSegment).getBytes("UTF-8"), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + defaultSegment.getIdentifier() + ) + ); + + // Insert second segment. + final SegmentPublishResult result2 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment2), + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new ObjectMetadata(ImmutableMap.of("foo", "baz")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(defaultSegment2), true), result2); + + Assert.assertArrayEquals( + mapper.writeValueAsString(defaultSegment2).getBytes("UTF-8"), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + defaultSegment2.getIdentifier() + ) + ); + + // Examine metadata. + Assert.assertEquals( + new ObjectMetadata(ImmutableMap.of("foo", "baz")), + coordinator.getDataSourceMetadata("fooDataSource") + ); + } + + @Test + public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException + { + final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + new ObjectMetadata(ImmutableMap.of("foo", "baz")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(), false), result1); + } + + @Test + public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException + { + final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + new ObjectMetadata(null), + new ObjectMetadata(ImmutableMap.of("foo", "baz")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(defaultSegment), true), result1); + + final SegmentPublishResult result2 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment2), + new ObjectMetadata(null), + new ObjectMetadata(ImmutableMap.of("foo", "baz")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(), false), result2); + } + + @Test + public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException + { + final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment), + new ObjectMetadata(null), + new ObjectMetadata(ImmutableMap.of("foo", "baz")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(defaultSegment), true), result1); + + final SegmentPublishResult result2 = coordinator.announceHistoricalSegments( + ImmutableSet.of(defaultSegment2), + new ObjectMetadata(ImmutableMap.of("foo", "qux")), + new ObjectMetadata(ImmutableMap.of("foo", "baz")) + ); + Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(), false), result2); + } + @Test public void testSimpleUsedList() throws IOException { @@ -191,7 +289,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 f22fb2c2cf896b4b436e718548b825d7496aa470 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 1 Mar 2016 16:51:50 -0800 Subject: [PATCH 6/6] KafkaIndexTask. Reads a specific offset range from specific partitions, and can use dataSource metadata transactions to guarantee exactly-once ingestion. Each task has a finite lifecycle, so it is expected that some process will be supervising existing tasks and creating new ones when needed. --- distribution/pom.xml | 2 + .../kafka-indexing-service/pom.xml | 91 ++ .../kafka/KafkaDataSourceMetadata.java | 123 ++ .../druid/indexing/kafka/KafkaIOConfig.java | 115 ++ .../druid/indexing/kafka/KafkaIndexTask.java | 587 ++++++++ .../indexing/kafka/KafkaIndexTaskModule.java | 51 + .../druid/indexing/kafka/KafkaPartitions.java | 97 ++ .../indexing/kafka/KafkaTuningConfig.java | 147 ++ .../io.druid.initialization.DruidModule | 1 + .../kafka/KafkaDataSourceMetadataTest.java | 101 ++ .../indexing/kafka/KafkaIndexTaskTest.java | 1244 +++++++++++++++++ .../druid/indexing/kafka/test/TestBroker.java | 114 ++ pom.xml | 1 + 13 files changed, 2674 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/pom.xml create mode 100644 extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaDataSourceMetadata.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaPartitions.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java create mode 100644 extensions-core/kafka-indexing-service/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaDataSourceMetadataTest.java create mode 100644 extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java create mode 100644 extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java diff --git a/distribution/pom.xml b/distribution/pom.xml index eb6d4e20f569..2e089af916c6 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -81,6 +81,8 @@ -c io.druid.extensions:druid-kafka-extraction-namespace -c + io.druid.extensions:druid-kafka-indexing-service + -c io.druid.extensions:mysql-metadata-storage -c io.druid.extensions:druid-namespace-lookup diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml new file mode 100644 index 000000000000..7b32b5bf48b0 --- /dev/null +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -0,0 +1,91 @@ + + + + + 4.0.0 + + io.druid.extensions + druid-kafka-indexing-service + druid-kafka-indexing-service + druid-kafka-indexing-service + + + io.druid + druid + 0.9.0-SNAPSHOT + ../../pom.xml + + + + + io.druid + druid-api + + + io.druid + druid-indexing-service + 0.9.0-SNAPSHOT + provided + + + org.apache.kafka + kafka-clients + 0.9.0.0 + + + + + junit + junit + test + + + org.apache.kafka + kafka_2.11 + 0.9.0.0 + test + + + io.druid + druid-server + 0.9.0-SNAPSHOT + test-jar + test + + + io.druid + druid-indexing-service + 0.9.0-SNAPSHOT + test-jar + test + + + org.apache.curator + curator-test + test + + + org.easymock + easymock + test + + + + diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaDataSourceMetadata.java new file mode 100644 index 000000000000..84b61dca2b09 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -0,0 +1,123 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; +import com.metamx.common.IAE; +import io.druid.indexing.overlord.DataSourceMetadata; + +import java.util.Map; +import java.util.Objects; + +public class KafkaDataSourceMetadata implements DataSourceMetadata +{ + private final KafkaPartitions kafkaPartitions; + + @JsonCreator + public KafkaDataSourceMetadata( + @JsonProperty("partitions") KafkaPartitions kafkaPartitions + ) + { + this.kafkaPartitions = kafkaPartitions; + } + + @JsonProperty("partitions") + public KafkaPartitions getKafkaPartitions() + { + return kafkaPartitions; + } + + @Override + public boolean isValidStart() + { + return true; + } + + @Override + public boolean matches(DataSourceMetadata other) + { + if (getClass() != other.getClass()) { + return false; + } + + return plus(other).equals(other.plus(this)); + } + + @Override + public DataSourceMetadata plus(DataSourceMetadata other) + { + if (!(other instanceof KafkaDataSourceMetadata)) { + throw new IAE( + "Expected instance of %s, got %s", + KafkaDataSourceMetadata.class.getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) other; + + if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) { + // Same topic, merge offsets. + final Map newMap = Maps.newHashMap(); + + for (Map.Entry entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) { + newMap.put(entry.getKey(), entry.getValue()); + } + + for (Map.Entry entry : that.getKafkaPartitions().getPartitionOffsetMap().entrySet()) { + newMap.put(entry.getKey(), entry.getValue()); + } + + return new KafkaDataSourceMetadata(new KafkaPartitions(kafkaPartitions.getTopic(), newMap)); + } else { + // Different topic, prefer "other". + return other; + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) o; + return Objects.equals(kafkaPartitions, that.kafkaPartitions); + } + + @Override + public int hashCode() + { + return Objects.hash(kafkaPartitions); + } + + @Override + public String toString() + { + return "KafkaDataSourceMetadata{" + + "kafkaPartitions=" + kafkaPartitions + + '}'; + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java new file mode 100644 index 000000000000..550465e6ef68 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIOConfig.java @@ -0,0 +1,115 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.druid.segment.indexing.IOConfig; + +import java.util.Map; + +public class KafkaIOConfig implements IOConfig +{ + private static final boolean DEFAULT_USE_TRANSACTION = true; + + private final String sequenceName; + private final KafkaPartitions startPartitions; + private final KafkaPartitions endPartitions; + private final Map consumerProperties; + private final boolean useTransaction; + + @JsonCreator + public KafkaIOConfig( + @JsonProperty("sequenceName") String sequenceName, + @JsonProperty("startPartitions") KafkaPartitions startPartitions, + @JsonProperty("endPartitions") KafkaPartitions endPartitions, + @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("useTransaction") Boolean useTransaction + ) + { + this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName"); + this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions"); + this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions"); + this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); + this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; + + Preconditions.checkArgument( + startPartitions.getTopic().equals(endPartitions.getTopic()), + "start topic and end topic must match" + ); + + Preconditions.checkArgument( + startPartitions.getPartitionOffsetMap().keySet().equals(endPartitions.getPartitionOffsetMap().keySet()), + "start partition set and end partition set must match" + ); + + for (int partition : endPartitions.getPartitionOffsetMap().keySet()) { + Preconditions.checkArgument( + endPartitions.getPartitionOffsetMap().get(partition) >= startPartitions.getPartitionOffsetMap() + .get(partition), + "end offset must be >= start offset for partition[%d]", + partition + ); + } + } + + @JsonProperty + public String getSequenceName() + { + return sequenceName; + } + + @JsonProperty + public KafkaPartitions getStartPartitions() + { + return startPartitions; + } + + @JsonProperty + public KafkaPartitions getEndPartitions() + { + return endPartitions; + } + + @JsonProperty + public Map getConsumerProperties() + { + return consumerProperties; + } + + @JsonProperty + public boolean isUseTransaction() + { + return useTransaction; + } + + @Override + public String toString() + { + return "KafkaIOConfig{" + + "sequenceName='" + sequenceName + '\'' + + ", startPartitions=" + startPartitions + + ", endPartitions=" + endPartitions + + ", consumerProperties=" + consumerProperties + + ", useTransaction=" + useTransaction + + '}'; + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java new file mode 100644 index 000000000000..6474ca5cd241 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -0,0 +1,587 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.metamx.common.ISE; +import com.metamx.common.RetryUtils; +import com.metamx.common.guava.Sequence; +import com.metamx.common.logger.Logger; +import com.metamx.common.parsers.ParseException; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; +import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentInsertAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.TaskResource; +import io.druid.query.DruidMetrics; +import io.druid.query.NoopQueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.realtime.FireDepartment; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.RealtimeMetricsMonitor; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.FiniteAppenderatorDriver; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import io.druid.timeline.DataSegment; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetOutOfRangeException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.Callable; + +public class KafkaIndexTask extends AbstractTask +{ + private static final Logger log = new Logger(KafkaIndexTask.class); + private static final String TYPE = "index_kafka"; + private static final Random RANDOM = new Random(); + private static final long POLL_TIMEOUT = 100; + private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; + + private final DataSchema dataSchema; + private final InputRowParser parser; + private final KafkaTuningConfig tuningConfig; + private final KafkaIOConfig ioConfig; + + private volatile Appenderator appenderator = null; + private volatile FireDepartmentMetrics fireDepartmentMetrics = null; + private volatile boolean startedReading = false; + private volatile boolean stopping = false; + private volatile boolean publishing = false; + private volatile Thread runThread = null; + + @JsonCreator + public KafkaIndexTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig, + @JsonProperty("ioConfig") KafkaIOConfig ioConfig, + @JsonProperty("context") Map context + ) + { + super( + id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt()) : id, + String.format("%s_%s", TYPE, dataSchema.getDataSource()), + taskResource, + dataSchema.getDataSource(), + context + ); + + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); + } + + private static String makeTaskId(String dataSource, 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 Joiner.on("_").join(TYPE, dataSource, suffix); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public KafkaTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty("ioConfig") + public KafkaIOConfig getIOConfig() + { + return ioConfig; + } + + /** + * Public for tests. + */ + @JsonIgnore + public boolean hasStartedReading() + { + return startedReading; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + log.info("Starting up!"); + + runThread = Thread.currentThread(); + + // Set up FireDepartmentMetrics + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ); + fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + toolbox.getMonitorScheduler().addMonitor( + new RealtimeMetricsMonitor( + ImmutableList.of(fireDepartmentForMetrics), + ImmutableMap.of(DruidMetrics.TASK_ID, new String[]{getId()}) + ) + ); + + try ( + final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); + final FiniteAppenderatorDriver driver = newDriver(appenderator0, toolbox); + final KafkaConsumer consumer = newConsumer() + ) { + appenderator = appenderator0; + + final String topic = ioConfig.getStartPartitions().getTopic(); + + // Start up, set up initial offsets. + final Object restoredMetadata = driver.startJob(); + final Map nextOffsets = Maps.newHashMap(); + if (restoredMetadata == null) { + nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap()); + } else { + final Map restoredMetadataMap = (Map) restoredMetadata; + final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), + KafkaPartitions.class + ); + nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); + + // Sanity checks. + if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) { + throw new ISE( + "WTF?! Restored topic[%s] but expected topic[%s]", + restoredNextPartitions.getTopic(), + ioConfig.getStartPartitions().getTopic() + ); + } + + if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) { + throw new ISE( + "WTF?! Restored partitions[%s] but expected partitions[%s]", + nextOffsets.keySet(), + ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() + ); + } + } + + // Set up committer. + final Supplier committerSupplier = new Supplier() + { + @Override + public Committer get() + { + final Map snapshot = ImmutableMap.copyOf(nextOffsets); + + return new Committer() + { + @Override + public Object getMetadata() + { + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new KafkaPartitions( + ioConfig.getStartPartitions().getTopic(), + snapshot + ) + ); + } + + @Override + public void run() + { + // Do nothing. + } + }; + } + }; + + // Initialize consumer assignment. + final Set assignment = Sets.newHashSet(); + for (Map.Entry entry : nextOffsets.entrySet()) { + final long endOffset = ioConfig.getEndPartitions().getPartitionOffsetMap().get(entry.getKey()); + if (entry.getValue() < endOffset) { + assignment.add(entry.getKey()); + } else if (entry.getValue() == endOffset) { + log.info("Finished reading partition[%d].", entry.getKey()); + } else { + throw new ISE( + "WTF?! Cannot start from offset[%,d] > endOffset[%,d]", + entry.getValue(), + endOffset + ); + } + } + + assignPartitions(consumer, topic, assignment); + + // Seek to starting offsets. + for (final int partition : assignment) { + final long offset = nextOffsets.get(partition); + log.info("Seeking partition[%d] to offset[%,d].", partition, offset); + consumer.seek(new TopicPartition(topic, partition), offset); + } + + // Main loop. + // Could eventually support early termination (triggered by a supervisor) + // Could eventually support leader/follower mode (for keeping replicas more in sync) + boolean stillReading = true; + while (stillReading) { + if (stopping) { + log.info("Stopping early."); + break; + } + + // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to + // offset is not present in the topic-partition. This can happen if we're asking a task to read from data + // that has not been written yet (which is totally legitimate). So let's wait for it to show up. + final ConsumerRecords records = RetryUtils.retry( + new Callable>() + { + @Override + public ConsumerRecords call() throws Exception + { + try { + return consumer.poll(POLL_TIMEOUT); + } + finally { + startedReading = true; + } + } + }, + new Predicate() + { + @Override + public boolean apply(Throwable input) + { + return input instanceof OffsetOutOfRangeException; + } + }, + Integer.MAX_VALUE + ); + + for (ConsumerRecord record : records) { + if (log.isTraceEnabled()) { + log.trace( + "Got topic[%s] partition[%d] offset[%,d].", + record.topic(), + record.partition(), + record.offset() + ); + } + + if (record.offset() < ioConfig.getEndPartitions().getPartitionOffsetMap().get(record.partition())) { + if (record.offset() != nextOffsets.get(record.partition())) { + throw new ISE( + "WTF?! Got offset[%,d] after offset[%,d] in partition[%d].", + record.offset(), + nextOffsets.get(record.partition()), + record.partition() + ); + } + + try { + final InputRow row = Preconditions.checkNotNull(parser.parse(ByteBuffer.wrap(record.value())), "row"); + final SegmentIdentifier identifier = driver.add(row, committerSupplier); + + if (identifier == null) { + // Failure to allocate segment puts determinism at risk, bail out to be safe. + // May want configurable behavior here at some point. + // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. + throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); + } + + fireDepartmentMetrics.incrementProcessed(); + } + catch (ParseException e) { + if (tuningConfig.isReportParseExceptions()) { + throw e; + } else { + log.debug( + e, + "Dropping unparseable row from partition[%d] offset[%,d].", + record.partition(), + record.offset() + ); + + fireDepartmentMetrics.incrementUnparseable(); + } + } + + final long nextOffset = record.offset() + 1; + final long endOffset = ioConfig.getEndPartitions().getPartitionOffsetMap().get(record.partition()); + + nextOffsets.put(record.partition(), nextOffset); + + if (nextOffset == endOffset && assignment.remove(record.partition())) { + log.info("Finished reading topic[%s], partition[%,d].", record.topic(), record.partition()); + assignPartitions(consumer, topic, assignment); + stillReading = !assignment.isEmpty(); + } + } + } + } + + // Persist pending data. + final Committer finalCommitter = committerSupplier.get(); + driver.persist(finalCommitter); + + publishing = true; + if (stopping) { + // Stopped gracefully. Exit code shouldn't matter, so fail to be on the safe side. + return TaskStatus.failure(getId()); + } + + final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() + { + @Override + public boolean publishSegments(Set segments, Object commitMetadata) throws IOException + { + // Sanity check, we should only be publishing things that match our desired end state. + if (!ioConfig.getEndPartitions().equals(((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS))) { + throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); + } + + final SegmentInsertAction action; + + if (ioConfig.isUseTransaction()) { + action = new SegmentInsertAction( + segments, + new KafkaDataSourceMetadata(ioConfig.getStartPartitions()), + new KafkaDataSourceMetadata(ioConfig.getEndPartitions()) + ); + } else { + action = new SegmentInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); + + return toolbox.getTaskActionClient().submit(action).isSuccess(); + } + }; + + final SegmentsAndMetadata published = driver.finish(publisher, committerSupplier.get()); + if (published == null) { + throw new ISE("Transaction failure publishing segments, aborting"); + } else { + log.info( + "Published segments[%s] with metadata[%s].", + Joiner.on(", ").join( + Iterables.transform( + published.getSegments(), + new Function() + { + @Override + public String apply(DataSegment input) + { + return input.getIdentifier(); + } + } + ) + ), + published.getCommitMetadata() + ); + } + } + + return success(); + } + + @Override + public boolean canRestore() + { + return true; + } + + @Override + public void stopGracefully() + { + log.info("Stopping gracefully."); + + stopping = true; + if (publishing && runThread.isAlive()) { + log.info("stopGracefully: Run thread started publishing, interrupting it."); + runThread.interrupt(); + } + } + + @Override + public QueryRunner getQueryRunner(Query query) + { + if (appenderator == null) { + // Not yet initialized, no data yet, just return a noop runner. + return new NoopQueryRunner<>(); + } + + return new QueryRunner() + { + @Override + public Sequence run(final Query query, final Map responseContext) + { + return query.run(appenderator, responseContext); + } + }; + } + + @VisibleForTesting + public FireDepartmentMetrics getFireDepartmentMetrics() + { + return fireDepartmentMetrics; + } + + private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) + { + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(new File(toolbox.getTaskWorkDir(), "persist")), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + tuningConfig.getBuildV9Directly() ? toolbox.getIndexMergerV9() : toolbox.getIndexMerger(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig() + ); + } + + private FiniteAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox + ) + { + return new FiniteAppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + ioConfig.getSequenceName() + ), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getObjectMapper(), + tuningConfig.getMaxRowsPerSegment(), + tuningConfig.getHandoffConditionTimeout() + ); + } + + private KafkaConsumer newConsumer() + { + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + + final Properties props = new Properties(); + + for (Map.Entry entry : ioConfig.getConsumerProperties().entrySet()) { + props.setProperty(entry.getKey(), entry.getValue()); + } + + props.setProperty("enable.auto.commit", "false"); + props.setProperty("auto.offset.reset", "none"); + props.setProperty("key.deserializer", ByteArrayDeserializer.class.getName()); + props.setProperty("value.deserializer", ByteArrayDeserializer.class.getName()); + + return new KafkaConsumer<>(props); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } + + private static void assignPartitions( + final KafkaConsumer consumer, + final String topic, + final Set partitions + ) + { + consumer.assign( + Lists.newArrayList( + Iterables.transform( + partitions, + new Function() + { + @Override + public TopicPartition apply(Integer n) + { + return new TopicPartition(topic, n); + } + } + ) + ) + ); + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java new file mode 100644 index 000000000000..cce67287be50 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -0,0 +1,51 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.kafka; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; + +import java.util.List; + +public class KafkaIndexTaskModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes( + new NamedType(KafkaIndexTask.class, "index_kafka"), + new NamedType(KafkaDataSourceMetadata.class, "kafka"), + new NamedType(KafkaIOConfig.class, "kafka"), + new NamedType(KafkaTuningConfig.class, "kafka") + ) + ); + } + + @Override + public void configure(Binder binder) + { + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaPartitions.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaPartitions.java new file mode 100644 index 000000000000..f0d7370bfc8f --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaPartitions.java @@ -0,0 +1,97 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Objects; + +public class KafkaPartitions +{ + private final String topic; + private final Map partitionOffsetMap; + + @JsonCreator + public KafkaPartitions( + @JsonProperty("topic") final String topic, + @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap + ) + { + this.topic = topic; + this.partitionOffsetMap = ImmutableMap.copyOf(partitionOffsetMap); + + // Validate partitionOffsetMap + for (Map.Entry entry : partitionOffsetMap.entrySet()) { + Preconditions.checkArgument( + entry.getValue() >= 0, + String.format( + "partition[%d] offset[%d] invalid", + entry.getKey(), + entry.getValue() + ) + ); + } + } + + @JsonProperty + public String getTopic() + { + return topic; + } + + @JsonProperty + public Map getPartitionOffsetMap() + { + return partitionOffsetMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaPartitions that = (KafkaPartitions) o; + return Objects.equals(topic, that.topic) && + Objects.equals(partitionOffsetMap, that.partitionOffsetMap); + } + + @Override + public int hashCode() + { + return Objects.hash(topic, partitionOffsetMap); + } + + @Override + public String toString() + { + return "KafkaPartitions{" + + "topic='" + topic + '\'' + + ", partitionOffsetMap=" + partitionOffsetMap + + '}'; + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java new file mode 100644 index 000000000000..374b2dec909c --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -0,0 +1,147 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.IndexSpec; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfig; +import io.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.joda.time.Period; + +import java.io.File; + +public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig +{ + private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; + + private final int maxRowsInMemory; + private final int maxRowsPerSegment; + private final Period intermediatePersistPeriod; + private final File basePersistDirectory; + private final int maxPendingPersists; + private final IndexSpec indexSpec; + private final boolean buildV9Directly; + private final boolean reportParseExceptions; + private final long handoffConditionTimeout; + + @JsonCreator + public KafkaTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout + ) + { + // Cannot be a static because default basePersistDirectory is unique per-instance + final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); + + this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; + this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + this.intermediatePersistPeriod = intermediatePersistPeriod == null + ? defaults.getIntermediatePersistPeriod() + : intermediatePersistPeriod; + this.basePersistDirectory = defaults.getBasePersistDirectory(); + this.maxPendingPersists = maxPendingPersists == null ? defaults.getMaxPendingPersists() : maxPendingPersists; + this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; + this.buildV9Directly = buildV9Directly == null ? defaults.getBuildV9Directly() : buildV9Directly; + this.reportParseExceptions = reportParseExceptions == null + ? defaults.isReportParseExceptions() + : reportParseExceptions; + this.handoffConditionTimeout = handoffConditionTimeout == null + ? defaults.getHandoffConditionTimeout() + : handoffConditionTimeout; + } + + @JsonProperty + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @JsonProperty + public int getMaxRowsPerSegment() + { + return maxRowsPerSegment; + } + + @JsonProperty + public Period getIntermediatePersistPeriod() + { + return intermediatePersistPeriod; + } + + @JsonProperty + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @JsonProperty + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @JsonProperty + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @JsonProperty + public boolean getBuildV9Directly() + { + return buildV9Directly; + } + + @JsonProperty + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @JsonProperty + public long getHandoffConditionTimeout() + { + return handoffConditionTimeout; + } + + public KafkaTuningConfig withBasePersistDirectory(File dir) + { + return new KafkaTuningConfig( + maxRowsInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + dir, + maxPendingPersists, + indexSpec, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout + ); + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/kafka-indexing-service/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..16aec94a8dfe --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.indexing.kafka.KafkaIndexTaskModule diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaDataSourceMetadataTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaDataSourceMetadataTest.java new file mode 100644 index 000000000000..32e9b08022c1 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaDataSourceMetadataTest.java @@ -0,0 +1,101 @@ +/* + * 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.kafka; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class KafkaDataSourceMetadataTest +{ + private static final KafkaDataSourceMetadata KM0 = KM("foo", ImmutableMap.of()); + private static final KafkaDataSourceMetadata KM1 = KM("foo", ImmutableMap.of(0, 2L, 1, 3L)); + private static final KafkaDataSourceMetadata KM2 = KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)); + private static final KafkaDataSourceMetadata KM3 = KM("foo", ImmutableMap.of(0, 2L, 2, 5L)); + + @Test + public void testMatches() + { + Assert.assertTrue(KM0.matches(KM0)); + Assert.assertTrue(KM0.matches(KM1)); + Assert.assertTrue(KM0.matches(KM2)); + Assert.assertTrue(KM0.matches(KM3)); + + Assert.assertTrue(KM1.matches(KM0)); + Assert.assertTrue(KM1.matches(KM1)); + Assert.assertFalse(KM1.matches(KM2)); + Assert.assertTrue(KM1.matches(KM3)); + + Assert.assertTrue(KM2.matches(KM0)); + Assert.assertFalse(KM2.matches(KM1)); + Assert.assertTrue(KM2.matches(KM2)); + Assert.assertTrue(KM2.matches(KM3)); + + Assert.assertTrue(KM3.matches(KM0)); + Assert.assertTrue(KM3.matches(KM1)); + Assert.assertTrue(KM3.matches(KM2)); + Assert.assertTrue(KM3.matches(KM3)); + } + + @Test + public void testIsValidStart() + { + Assert.assertTrue(KM0.isValidStart()); + Assert.assertTrue(KM1.isValidStart()); + Assert.assertTrue(KM2.isValidStart()); + Assert.assertTrue(KM3.isValidStart()); + } + + @Test + public void testPlus() + { + Assert.assertEquals( + KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), + KM1.plus(KM3) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), + KM0.plus(KM2) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), + KM1.plus(KM2) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), + KM2.plus(KM1) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), + KM2.plus(KM2) + ); + } + + private static KafkaDataSourceMetadata KM(String topic, Map offsets) + { + return new KafkaDataSourceMetadata(new KafkaPartitions(topic, offsets)); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java new file mode 100644 index 000000000000..70911cc3dd73 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -0,0 +1,1244 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; +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.io.Files; +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.CompressionUtils; +import com.metamx.common.Granularity; +import com.metamx.common.ISE; +import com.metamx.common.guava.Sequences; +import com.metamx.common.logger.Logger; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.MonitorScheduler; +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; +import io.druid.data.input.impl.JSONPathFieldSpec; +import io.druid.data.input.impl.JSONPathSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.indexing.common.SegmentLoaderFactory; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.TestUtils; +import io.druid.indexing.common.actions.LocalTaskActionClientFactory; +import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.actions.TaskActionToolbox; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.kafka.test.TestBroker; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.MetadataTaskStorage; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.test.TestDataSegmentAnnouncer; +import io.druid.indexing.test.TestDataSegmentKiller; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.metadata.EntryExistsException; +import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; +import io.druid.metadata.TestDerbyConnector; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; +import io.druid.query.Druids; +import io.druid.query.IntervalChunkingQueryRunnerDecorator; +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.QueryWatcher; +import io.druid.query.Result; +import io.druid.query.SegmentDescriptor; +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.query.timeseries.TimeseriesResultValue; +import io.druid.segment.IndexIO; +import io.druid.segment.QueryableIndex; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPusherConfig; +import io.druid.segment.loading.SegmentLoaderConfig; +import io.druid.segment.loading.SegmentLoaderLocalCacheManager; +import io.druid.segment.loading.StorageLocationConfig; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.timeline.DataSegment; +import org.apache.curator.test.TestingCluster; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +@RunWith(Parameterized.class) +public class KafkaIndexTaskTest +{ + private final boolean buildV9Directly; + private long handoffConditionTimeout = 0; + private boolean reportParseExceptions = false; + private boolean doHandoff = true; + + private TestingCluster zkServer; + private TestBroker kafkaServer; + private ServiceEmitter emitter; + private ListeningExecutorService taskExec; + private TaskToolboxFactory toolboxFactory; + private IndexerMetadataStorageCoordinator metadataStorageCoordinator; + private TaskStorage taskStorage; + private TaskLockbox taskLockbox; + private File directory; + + private final List runningTasks = Lists.newArrayList(); + + private static final Logger log = new Logger(KafkaIndexTaskTest.class); + + private static final DataSchema DATA_SCHEMA; + + private static final List> RECORDS = ImmutableList.of( + new ProducerRecord("topic0", 0, null, JB("2008", "a", "y", 1.0f)), + new ProducerRecord("topic0", 0, null, JB("2009", "b", "y", 1.0f)), + new ProducerRecord("topic0", 0, null, JB("2010", "c", "y", 1.0f)), + new ProducerRecord("topic0", 0, null, JB("2011", "d", "y", 1.0f)), + new ProducerRecord("topic0", 0, null, JB("2011", "e", "y", 1.0f)), + new ProducerRecord("topic0", 0, null, "unparseable".getBytes()), + new ProducerRecord("topic0", 0, null, JB("2013", "f", "y", 1.0f)), + new ProducerRecord("topic0", 1, null, JB("2012", "g", "y", 1.0f)) + ); + + static { + ObjectMapper objectMapper = new DefaultObjectMapper(); + DATA_SCHEMA = new DataSchema( + "test_ds", + objectMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + ImmutableList.of("dim1", "dim2"), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + Charsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec(Granularity.DAY, QueryGranularity.NONE, null), + objectMapper + ); + } + + @Parameterized.Parameters(name = "buildV9Directly = {0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of(new Object[]{true}, new Object[]{false}); + } + + public KafkaIndexTaskTest(boolean buildV9Directly) + { + this.buildV9Directly = buildV9Directly; + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + + @Before + public void setUp() throws Exception + { + emitter = new ServiceEmitter( + "service", + "host", + new LoggingEmitter( + log, + LoggingEmitter.Level.ERROR, + new DefaultObjectMapper() + ) + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + + makeToolboxFactory(); + + zkServer = new TestingCluster(1); + zkServer.start(); + + kafkaServer = new TestBroker( + zkServer.getConnectString(), + tempFolder.newFolder(), + 1, + ImmutableMap.of("num.partitions", "2") + ); + kafkaServer.start(); + + taskExec = MoreExecutors.listeningDecorator( + Executors.newCachedThreadPool( + Execs.makeThreadFactory("kafka-task-test-%d") + ) + ); + + handoffConditionTimeout = 0; + reportParseExceptions = false; + doHandoff = true; + } + + @After + public void tearDown() throws Exception + { + emitter.close(); + + synchronized (runningTasks) { + for (Task task : runningTasks) { + task.stopGracefully(); + } + + runningTasks.clear(); + } + + taskExec.shutdown(); + taskExec.awaitTermination(9999, TimeUnit.DAYS); + + kafkaServer.close(); + kafkaServer = null; + + zkServer.stop(); + zkServer = null; + + destroyToolboxFactory(); + } + + @Test(timeout = 60_000L) + public void testRunAfterDataInserted() throws Exception + { + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + @Test(timeout = 60_000L) + public void testRunBeforeDataInserted() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (!task.hasStartedReading()) { + Thread.sleep(10); + } + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + @Test(timeout = 60_000L) + public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception + { + handoffConditionTimeout = 5_000; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + @Test(timeout = 60_000L) + public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception + { + doHandoff = false; + handoffConditionTimeout = 100; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + @Test(timeout = 60_000L) + public void testReportParseExceptions() throws Exception + { + reportParseExceptions = true; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + } + + @Test(timeout = 60_000L) + public void testRunReplicas() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future1 = runTask(task1); + final ListenableFuture future2 = runTask(task2); + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + // Wait for tasks to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + @Test(timeout = 60_000L) + public void testRunConflicting() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIOConfig( + "sequence1", + new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + // Run first task + final ListenableFuture future1 = runTask(task1); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + + // Run second task + final ListenableFuture future2 = runTask(task2); + Assert.assertEquals(TaskStatus.Status.FAILED, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + + // Check published segments & metadata, should all be from the first task + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + @Test(timeout = 60_000L) + public void testRunConflictingWithoutTransactions() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + false + ), + null + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIOConfig( + "sequence1", + new KafkaPartitions("topic0", ImmutableMap.of(0, 3L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 7L)), + kafkaServer.consumerProperties(), + false + ), + null + ); + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + // Run first task + final ListenableFuture future1 = runTask(task1); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + // Run second task + final ListenableFuture future2 = runTask(task2); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); + SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentDim1(desc4)); + } + + @Test(timeout = 60_000L) + public void testRunOneTaskTwoPartitions() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L, 1, 0L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 1L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + // Wait for tasks to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(4, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2012/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 1L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc3)); + } + + @Test(timeout = 60_000L) + public void testRunTwoTasksTwoPartitions() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIOConfig( + "sequence1", + new KafkaPartitions("topic0", ImmutableMap.of(1, 0L)), + new KafkaPartitions("topic0", ImmutableMap.of(1, 1L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future1 = runTask(task1); + final ListenableFuture future2 = runTask(task2); + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : RECORDS) { + kafkaProducer.send(record).get(); + } + } + + // Wait for tasks to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L, 1, 1L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc3)); + } + + @Test(timeout = 60_000L) + public void testRestore() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future1 = runTask(task1); + + // Insert some data, but not enough for the task to finish + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { + kafkaProducer.send(record).get(); + } + } + + while (countEvents(task1) != 2) { + Thread.sleep(25); + } + + Assert.assertEquals(2, countEvents(task1)); + + // Stop gracefully + task1.stopGracefully(); + Assert.assertEquals(TaskStatus.Status.FAILED, future1.get().getStatusCode()); + + // Start a new task + final KafkaIndexTask task2 = createTask( + task1.getId(), + new KafkaIOConfig( + "sequence0", + new KafkaPartitions("topic0", ImmutableMap.of(0, 2L)), + new KafkaPartitions("topic0", ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true + ), + null + ); + + final ListenableFuture future2 = runTask(task2); + + // Insert remaining data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { + kafkaProducer.send(record).get(); + } + } + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(2, task1.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions("topic0", ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + } + + private ListenableFuture runTask(final Task task) + { + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + catch (EntryExistsException e) { + // suppress + } + taskLockbox.syncFromStorage(); + final TaskToolbox toolbox = toolboxFactory.build(task); + synchronized (runningTasks) { + runningTasks.add(task); + } + return taskExec.submit( + new Callable() + { + @Override + public TaskStatus call() throws Exception + { + try { + if (task.isReady(toolbox.getTaskActionClient())) { + return task.run(toolbox); + } else { + throw new ISE("Task is not ready"); + } + } + catch (Exception e) { + log.warn(e, "Task failed"); + return TaskStatus.failure(task.getId()); + } + } + } + ); + } + + private TaskLock getLock(final Task task, final Interval interval) + { + return Iterables.find( + taskLockbox.findLocksForTask(task), + new Predicate() + { + @Override + public boolean apply(TaskLock lock) + { + return lock.getInterval().contains(interval); + } + } + ); + } + + private KafkaIndexTask createTask( + final String taskId, + final KafkaIOConfig ioConfig, + final Integer maxRowsPerSegment + ) + { + final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( + 1000, + maxRowsPerSegment, + new Period("P1Y"), + null, + null, + null, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout + ); + return new KafkaIndexTask( + taskId, + null, + DATA_SCHEMA, + tuningConfig, + ioConfig, + null + ); + } + + private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() + { + return new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>of( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + new IntervalChunkingQueryRunnerDecorator(null, null, null) + { + @Override + public QueryRunner decorate( + QueryRunner delegate, QueryToolChest> toolChest + ) + { + return delegate; + } + } + ), + new TimeseriesQueryEngine(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + // do nothing + } + } + ) + ) + ); + } + + private void makeToolboxFactory() throws IOException + { + directory = tempFolder.newFolder(); + final TestUtils testUtils = new TestUtils(); + final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + for (Module module : new KafkaIndexTaskModule().getJacksonModules()) { + objectMapper.registerModule(module); + } + final TaskConfig taskConfig = new TaskConfig( + new File(directory, "taskBaseDir").getPath(), + null, + null, + 50000, + null, + false, + null, + null + ); + final TestDerbyConnector derbyConnector = derby.getConnector(); + derbyConnector.createDataSourceTable(); + derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentTable(); + derbyConnector.createRulesTable(); + derbyConnector.createConfigTable(); + derbyConnector.createTaskTables(); + derbyConnector.createAuditTable(); + taskStorage = new MetadataTaskStorage( + derbyConnector, + new TaskStorageConfig(null), + new SQLMetadataStorageActionHandlerFactory( + derbyConnector, + derby.metadataTablesConfigSupplier().get(), + objectMapper + ) + ); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + testUtils.getTestObjectMapper(), + derby.metadataTablesConfigSupplier().get(), + derbyConnector + ); + taskLockbox = new TaskLockbox(taskStorage); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( + taskLockbox, + metadataStorageCoordinator, + emitter + ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( + taskStorage, + taskActionToolbox + ); + final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory() + { + @Override + public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) + { + return new SegmentHandoffNotifier() + { + @Override + public boolean registerSegmentHandoffCallback( + SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable + ) + { + if (doHandoff) { + // Simulate immediate handoff + exec.execute(handOffRunnable); + } + return true; + } + + @Override + public void start() + { + //Noop + } + + @Override + public void close() + { + //Noop + } + }; + } + }; + final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); + dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); + final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper); + toolboxFactory = new TaskToolboxFactory( + taskConfig, + taskActionClientFactory, + emitter, + dataSegmentPusher, + new TestDataSegmentKiller(), + null, // DataSegmentMover + null, // DataSegmentArchiver + new TestDataSegmentAnnouncer(), + handoffNotifierFactory, + makeTimeseriesOnlyConglomerate(), + MoreExecutors.sameThreadExecutor(), // queryExecutorService + EasyMock.createMock(MonitorScheduler.class), + new SegmentLoaderFactory( + new SegmentLoaderLocalCacheManager( + null, + new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }, testUtils.getTestObjectMapper() + ) + ), + testUtils.getTestObjectMapper(), + testUtils.getTestIndexMerger(), + testUtils.getTestIndexIO(), + MapCache.create(1024), + new CacheConfig(), + testUtils.getTestIndexMergerV9() + ); + } + + private void destroyToolboxFactory() + { + toolboxFactory = null; + taskStorage = null; + taskLockbox = null; + metadataStorageCoordinator = null; + } + + private Set publishedDescriptors() throws IOException + { + return FluentIterable.from( + metadataStorageCoordinator.getUsedSegmentsForInterval( + DATA_SCHEMA.getDataSource(), + new Interval("0000/3000") + ) + ).transform( + new Function() + { + @Override + public SegmentDescriptor apply(DataSegment input) + { + return input.toDescriptor(); + } + } + ).toSet(); + } + + private File getSegmentDirectory() + { + return new File(directory, "segments"); + } + + private List readSegmentDim1(final SegmentDescriptor descriptor) throws IOException + { + File indexZip = new File( + String.format( + "%s/%s/%s_%s/%s/%d/index.zip", + getSegmentDirectory(), + DATA_SCHEMA.getDataSource(), + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + File outputLocation = new File( + directory, + String.format( + "%s_%s_%s_%s", + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + outputLocation.mkdir(); + CompressionUtils.unzip( + Files.asByteSource(indexZip), + outputLocation, + Predicates.alwaysFalse(), + false + ); + IndexIO indexIO = new TestUtils().getTestIndexIO(); + QueryableIndex index = indexIO.loadIndex(outputLocation); + DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding(); + List values = Lists.newArrayList(); + for (int i = 0; i < dim1.length(); i++) { + int id = dim1.getSingleValueRow(i); + String value = dim1.lookupName(id); + values.add(value); + } + return values; + } + + public long countEvents(final Task task) throws Exception + { + // Do a query. + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SCHEMA.getDataSource()) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("rows", "rows") + ) + ).granularity(QueryGranularity.ALL) + .intervals("0000/3000") + .build(); + + ArrayList> results = Sequences.toList( + task.getQueryRunner(query).run(query, ImmutableMap.of()), + Lists.>newArrayList() + ); + + return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric("rows"); + } + + private static byte[] JB(String timestamp, String dim1, String dim2, double met1) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.of("timestamp", timestamp, "dim1", dim1, "dim2", dim2, "met1", met1) + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) + { + final Interval interval = new Interval(intervalString); + return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java new file mode 100644 index 000000000000..9a98d430359d --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/test/TestBroker.java @@ -0,0 +1,114 @@ +/* + * 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.kafka.test; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServer; +import kafka.utils.SystemTime$; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import scala.Some; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.Map; +import java.util.Properties; +import java.util.Random; + +public class TestBroker implements Closeable +{ + private final String zookeeperConnect; + private final File directory; + private final int id; + private final Map brokerProps; + + private volatile KafkaServer server; + + public TestBroker(String zookeeperConnect, File directory, int id, Map brokerProps) + { + this.zookeeperConnect = zookeeperConnect; + this.directory = directory; + this.id = id; + this.brokerProps = brokerProps == null ? ImmutableMap.of() : brokerProps; + } + + public void start() + { + final Properties props = new Properties(); + props.setProperty("zookeeper.connect", zookeeperConnect); + props.setProperty("log.dirs", directory.toString()); + props.setProperty("broker.id", String.valueOf(id)); + props.setProperty("port", String.valueOf(new Random().nextInt(9999) + 10000)); + props.putAll(brokerProps); + + final KafkaConfig config = new KafkaConfig(props); + + server = new KafkaServer(config, SystemTime$.MODULE$, Some.apply(String.format("TestingBroker[%d]-", id))); + server.startup(); + } + + public int getPort() + { + return server.socketServer().config().port(); + } + + public KafkaProducer newProducer() + { + return new KafkaProducer(producerProperties()); + } + + public KafkaConsumer newConsumer() + { + return new KafkaConsumer(consumerProperties()); + } + + public Map producerProperties() + { + final Map props = Maps.newHashMap(); + props.put("bootstrap.servers", String.format("localhost:%d", getPort())); + props.put("key.serializer", ByteArraySerializer.class.getName()); + props.put("value.serializer", ByteArraySerializer.class.getName()); + props.put("acks", "all"); + return props; + } + + public Map consumerProperties() + { + final Map props = Maps.newHashMap(); + props.put("bootstrap.servers", String.format("localhost:%d", getPort())); + props.put("key.deserializer", ByteArrayDeserializer.class.getName()); + props.put("value.deserializer", ByteArrayDeserializer.class.getName()); + return props; + } + + @Override + public void close() throws IOException + { + if (server != null) { + server.shutdown(); + server.awaitShutdown(); + } + } +} diff --git a/pom.xml b/pom.xml index a70e710d8b58..a3c857fe5662 100644 --- a/pom.xml +++ b/pom.xml @@ -88,6 +88,7 @@ extensions-core/histogram extensions-core/kafka-eight extensions-core/kafka-extraction-namespace + extensions-core/kafka-indexing-service extensions-core/mysql-metadata-storage extensions-core/postgresql-metadata-storage extensions-core/namespace-lookup