From 1f7a089fa8d424cc3b77600dbd12e40451a815e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 19 Nov 2013 13:30:07 -0800 Subject: [PATCH 01/38] S3 storage: enable archiving in addition to deleting unused segments --- .../druid/storage/s3/S3DataSegmentKiller.java | 36 ++++++++++++++++--- .../storage/s3/S3DataSegmentKillerConfig.java | 22 ++++++++++++ .../storage/s3/S3StorageDruidModule.java | 1 + 3 files changed, 54 insertions(+), 5 deletions(-) create mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java index b9a44f631c6..cc9afbb18d9 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java @@ -27,6 +27,7 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Object; import java.util.Map; @@ -37,13 +38,16 @@ public class S3DataSegmentKiller implements DataSegmentKiller private static final Logger log = new Logger(S3DataSegmentKiller.class); private final RestS3Service s3Client; + private final S3DataSegmentKillerConfig config; @Inject public S3DataSegmentKiller( - RestS3Service s3Client + RestS3Service s3Client, + S3DataSegmentKillerConfig config ) { this.s3Client = s3Client; + this.config = config; } @Override @@ -54,14 +58,36 @@ public class S3DataSegmentKiller implements DataSegmentKiller String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + final String s3ArchiveBucket = config.getArchiveBucket(); if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); - s3Client.deleteObject(s3Bucket, s3Path); + if (config.isArchive()) { + log.info("Archiving index file[s3://%s/%s] to [s3://%s/%s]", + s3Bucket, + s3Path, + s3ArchiveBucket, + s3Path + ); + s3Client.moveObject(s3Bucket, s3Path, s3ArchiveBucket, new S3Object(s3Path), false); + } else { + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); + } } if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); - s3Client.deleteObject(s3Bucket, s3DescriptorPath); + if (config.isArchive()) { + log.info( + "Archiving descriptor file[s3://%s/%s] to [s3://%s/%s]", + s3Bucket, + s3DescriptorPath, + s3ArchiveBucket, + s3DescriptorPath + ); + s3Client.moveObject(s3Bucket, s3DescriptorPath, s3ArchiveBucket, new S3Object(s3DescriptorPath), false); + } else { + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); + } } } catch (ServiceException e) { diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java new file mode 100644 index 00000000000..b17e87ac2be --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java @@ -0,0 +1,22 @@ +package io.druid.storage.s3; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class S3DataSegmentKillerConfig +{ + @JsonProperty + public boolean archive = false; + + @JsonProperty + public String archiveBucket = ""; + + public boolean isArchive() + { + return archive; + } + + public String getArchiveBucket() + { + return archiveBucket; + } +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index f2675251f19..6bb4624a3d3 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -53,6 +53,7 @@ public class S3StorageDruidModule implements DruidModule Binders.dataSegmentKillerBinder(binder).addBinding("s3_zip").to(S3DataSegmentKiller.class).in(LazySingleton.class); Binders.dataSegmentPusherBinder(binder).addBinding("s3").to(S3DataSegmentPusher.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentPusherConfig.class); + JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentKillerConfig.class); Binders.taskLogsBinder(binder).addBinding("s3").to(S3TaskLogs.class); JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); From e38f2877fbb085e5e87706b9aeb0417315d9fd6b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 19 Nov 2013 14:14:52 -0800 Subject: [PATCH 02/38] default to archiving segments + docs --- docs/content/Configuration.md | 2 ++ .../main/java/io/druid/storage/s3/S3DataSegmentKiller.java | 6 ++++++ .../java/io/druid/storage/s3/S3DataSegmentKillerConfig.java | 2 +- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index d7e010b3006..e5c64c74c17 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -287,6 +287,8 @@ This deep storage is used to interface with Amazon's S3. |`druid.storage.bucket`|S3 bucket name.|none| |`druid.storage.basekey`|S3 base key.|none| |`druid.storage.disableAcl`|Boolean flag for ACL.|false| +|`druid.storage.archive`|Boolean flag. Archives killed segments instead of deleting them from S3|true| +|`druid.storage.archiveBucket`|S3 bucket name to archive segments to|none| #### HDFS Deep Storage diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java index cc9afbb18d9..5409568a917 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java @@ -58,8 +58,14 @@ public class S3DataSegmentKiller implements DataSegmentKiller String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + final String s3ArchiveBucket = config.getArchiveBucket(); + if(config.isArchive() && s3ArchiveBucket.isEmpty()) { + log.warn("S3 archive bucket not specified, refusing to delete segment [s3://%s/%s]", s3Bucket, s3Path); + return; + } + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { if (config.isArchive()) { log.info("Archiving index file[s3://%s/%s] to [s3://%s/%s]", diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java index b17e87ac2be..bd169b05f2b 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java @@ -5,7 +5,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class S3DataSegmentKillerConfig { @JsonProperty - public boolean archive = false; + public boolean archive = true; @JsonProperty public String archiveBucket = ""; From a417cd5df2c1b9121c067ee17140fb96ca9a590b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 5 Dec 2013 17:37:03 -0800 Subject: [PATCH 03/38] add archive task --- .../io/druid/indexing/common/TaskToolbox.java | 9 ++ .../indexing/common/TaskToolboxFactory.java | 6 +- .../common/actions/SegmentMoveAction.java | 77 ++++++++++++ .../indexing/common/actions/TaskAction.java | 3 +- .../druid/indexing/common/task/MoveTask.java | 110 ++++++++++++++++++ .../io/druid/indexing/common/task/Task.java | 1 + .../overlord/IndexerDBCoordinator.java | 39 ++++++- .../indexing/overlord/TaskLifecycleTest.java | 9 ++ .../worker/WorkerTaskMonitorTest.java | 4 +- .../druid/storage/s3/S3DataSegmentKiller.java | 41 +------ .../storage/s3/S3DataSegmentKillerConfig.java | 22 ---- .../druid/storage/s3/S3DataSegmentMover.java | 99 ++++++++++++++++ .../storage/s3/S3DataSegmentMoverConfig.java | 33 ++++++ .../storage/s3/S3StorageDruidModule.java | 3 +- .../segment/loading/OmniDataSegmentMover.java | 57 +++++++++ .../src/main/java/io/druid/cli/CliPeon.java | 4 + 16 files changed, 451 insertions(+), 66 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java delete mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java create mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java create mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java create mode 100644 server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java 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 0a7a505d4ec..56f08f3c9f3 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 @@ -30,6 +30,7 @@ import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.SegmentLoader; import io.druid.segment.loading.SegmentLoadingException; @@ -52,6 +53,7 @@ public class TaskToolbox private final ServiceEmitter emitter; private final DataSegmentPusher segmentPusher; private final DataSegmentKiller dataSegmentKiller; + private final DataSegmentMover dataSegmentMover; private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; @@ -68,6 +70,7 @@ public class TaskToolbox ServiceEmitter emitter, DataSegmentPusher segmentPusher, DataSegmentKiller dataSegmentKiller, + DataSegmentMover dataSegmentMover, DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, @@ -84,6 +87,7 @@ public class TaskToolbox this.emitter = emitter; this.segmentPusher = segmentPusher; this.dataSegmentKiller = dataSegmentKiller; + this.dataSegmentMover = dataSegmentMover; this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; @@ -119,6 +123,11 @@ public class TaskToolbox return dataSegmentKiller; } + public DataSegmentMover getDataSegmentMover() + { + return dataSegmentMover; + } + public DataSegmentAnnouncer getSegmentAnnouncer() { return segmentAnnouncer; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index ca00dccaf91..cf9614c4bd6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -24,13 +24,13 @@ import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.ServerView; -import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Processing; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPusher; import io.druid.server.coordination.DataSegmentAnnouncer; @@ -47,6 +47,7 @@ public class TaskToolboxFactory private final ServiceEmitter emitter; private final DataSegmentPusher segmentPusher; private final DataSegmentKiller dataSegmentKiller; + private final DataSegmentMover dataSegmentMover; private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; @@ -62,6 +63,7 @@ public class TaskToolboxFactory ServiceEmitter emitter, DataSegmentPusher segmentPusher, DataSegmentKiller dataSegmentKiller, + DataSegmentMover dataSegmentMover, DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, @@ -76,6 +78,7 @@ public class TaskToolboxFactory this.emitter = emitter; this.segmentPusher = segmentPusher; this.dataSegmentKiller = dataSegmentKiller; + this.dataSegmentMover = dataSegmentMover; this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; @@ -96,6 +99,7 @@ public class TaskToolboxFactory emitter, segmentPusher, dataSegmentKiller, + dataSegmentMover, segmentAnnouncer, newSegmentServerView, queryRunnerFactoryConglomerate, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java new file mode 100644 index 00000000000..67db4fc79be --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java @@ -0,0 +1,77 @@ +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.common.ISE; +import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.indexing.common.task.Task; +import io.druid.timeline.DataSegment; + +import java.io.IOException; +import java.util.Set; + +public class SegmentMoveAction implements TaskAction +{ + @JsonIgnore + private final Set segments; + + @JsonCreator + public SegmentMoveAction( + @JsonProperty("segments") Set segments + ) + { + this.segments = ImmutableSet.copyOf(segments); + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Void perform( + Task task, TaskActionToolbox toolbox + ) throws IOException + { + if(!toolbox.taskLockCoversSegments(task, segments, true)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); + } + + toolbox.getIndexerDBCoordinator().moveSegments(segments); + + // Emit metrics + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() + .setUser2(task.getDataSource()) + .setUser4(task.getType()); + + for (DataSegment segment : segments) { + metricBuilder.setUser5(segment.getInterval().toString()); + toolbox.getEmitter().emit(metricBuilder.build("indexer/segmentMoved/bytes", segment.getSize())); + } + + return null; + } + + @Override + public boolean isAudited() + { + return true; + } + + @Override + public String toString() + { + return "SegmentMoveAction{" + + "segments=" + segments + + '}'; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index d9bdfe5b694..1d59e40c6c3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -35,7 +35,8 @@ import java.io.IOException; @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class), @JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class), - @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class) + @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), + @JsonSubTypes.Type(name = "segmentMove", value = SegmentMoveAction.class) }) public interface TaskAction { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java new file mode 100644 index 00000000000..4ca600f8ea5 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -0,0 +1,110 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentListUnusedAction; +import io.druid.indexing.common.actions.SegmentMoveAction; +import io.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.List; + +public class MoveTask extends AbstractTask +{ + private static final Logger log = new Logger(MoveTask.class); + + @JsonCreator + public MoveTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + super( + TaskUtils.makeId(id, "move", dataSource, interval), + dataSource, + interval + ); + } + + @Override + public String getType() + { + return "move"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + // Confirm we have a lock (will throw if there isn't exactly one element) + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + + if(!myLock.getDataSource().equals(getDataSource())) { + throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); + } + + if(!myLock.getInterval().equals(getImplicitLockInterval().get())) { + throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getImplicitLockInterval().get()); + } + + // List unused segments + final List unusedSegments = toolbox + .getTaskActionClient() + .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); + + // Verify none of these segments have versions > lock version + for(final DataSegment unusedSegment : unusedSegments) { + if(unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { + throw new ISE( + "WTF?! Unused segment[%s] has version[%s] > task version[%s]", + unusedSegment.getIdentifier(), + unusedSegment.getVersion(), + myLock.getVersion() + ); + } + + log.info("OK to move segment: %s", unusedSegment.getIdentifier()); + } + + List movedSegments = Lists.newLinkedList(); + + // Move segments + for (DataSegment segment : unusedSegments) { + movedSegments.add(toolbox.getDataSegmentMover().move(segment)); + } + + // Update metadata for moved segments + toolbox.getTaskActionClient().submit(new SegmentMoveAction( + ImmutableSet.copyOf(movedSegments) + )); + + return TaskStatus.success(getId()); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 4d6afd2ebf6..32747bdc7d3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -45,6 +45,7 @@ import io.druid.query.QueryRunner; @JsonSubTypes.Type(name = "merge", value = MergeTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), + @JsonSubTypes.Type(name = "move", value = MoveTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java index 2a4b5e8912d..943e63fa821 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java @@ -28,8 +28,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import io.druid.db.DbConnector; -import io.druid.db.DbConnectorConfig; import io.druid.db.DbTablesConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; @@ -213,6 +211,24 @@ public class IndexerDBCoordinator return true; } + public void moveSegments(final Set segments) throws IOException + { + dbi.inTransaction( + new TransactionCallback() + { + @Override + public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception + { + for(final DataSegment segment : segments) { + moveSegment(handle, segment); + } + + return null; + } + } + ); + } + public void deleteSegments(final Set segments) throws IOException { dbi.inTransaction( @@ -235,10 +251,27 @@ public class IndexerDBCoordinator { handle.createStatement( String.format("DELETE from %s WHERE id = :id", dbTables.getSegmentsTable()) - ).bind("id", segment.getIdentifier()) + ) + .bind("id", segment.getIdentifier()) .execute(); } + private void moveSegment(final Handle handle, final DataSegment segment) throws IOException + { + try { + handle.createStatement( + String.format("UPDATE %s SET payload = :payload WHERE id = :id", dbTables.getSegmentsTable()) + ) + .bind("id", segment.getIdentifier()) + .bind("payload", jsonMapper.writeValueAsString(segment)) + .execute(); + } + catch (IOException e) { + log.error(e, "Exception inserting into DB"); + throw e; + } + } + public List getUnusedSegmentsForInterval(final String dataSource, final Interval interval) { List matchingSegments = dbi.withHandle( 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 632384ceb3d..fcbfed7589b 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 @@ -65,6 +65,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPuller; @@ -158,6 +159,14 @@ public class TaskLifecycleTest } }, + new DataSegmentMover() + { + @Override + public DataSegment move(DataSegment dataSegment) throws SegmentLoadingException + { + return dataSegment; + } + }, null, // segment announcer null, // new segment server view null, // query runner factory conglomerate corporation unionized collective diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index f80ca3cd8db..7723013d239 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -122,7 +122,7 @@ public class WorkerTaskMonitorTest new ThreadPoolTaskRunner( new TaskToolboxFactory( new TaskConfig(tmp.toString(), null, null, 0), - null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory( + null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory( new OmniSegmentLoader( ImmutableMap.of( "local", @@ -209,4 +209,4 @@ public class WorkerTaskMonitorTest Assert.assertEquals(task.getId(), taskAnnouncement.getTaskStatus().getId()); Assert.assertEquals(TaskStatus.Status.RUNNING, taskAnnouncement.getTaskStatus().getStatusCode()); } -} \ No newline at end of file +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java index 5409568a917..4e295d17aa4 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java @@ -38,16 +38,13 @@ public class S3DataSegmentKiller implements DataSegmentKiller private static final Logger log = new Logger(S3DataSegmentKiller.class); private final RestS3Service s3Client; - private final S3DataSegmentKillerConfig config; @Inject public S3DataSegmentKiller( - RestS3Service s3Client, - S3DataSegmentKillerConfig config + RestS3Service s3Client ) { this.s3Client = s3Client; - this.config = config; } @Override @@ -59,41 +56,13 @@ public class S3DataSegmentKiller implements DataSegmentKiller String s3Path = MapUtils.getString(loadSpec, "key"); String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; - final String s3ArchiveBucket = config.getArchiveBucket(); - - if(config.isArchive() && s3ArchiveBucket.isEmpty()) { - log.warn("S3 archive bucket not specified, refusing to delete segment [s3://%s/%s]", s3Bucket, s3Path); - return; - } - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - if (config.isArchive()) { - log.info("Archiving index file[s3://%s/%s] to [s3://%s/%s]", - s3Bucket, - s3Path, - s3ArchiveBucket, - s3Path - ); - s3Client.moveObject(s3Bucket, s3Path, s3ArchiveBucket, new S3Object(s3Path), false); - } else { - log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); - s3Client.deleteObject(s3Bucket, s3Path); - } + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); } if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - if (config.isArchive()) { - log.info( - "Archiving descriptor file[s3://%s/%s] to [s3://%s/%s]", - s3Bucket, - s3DescriptorPath, - s3ArchiveBucket, - s3DescriptorPath - ); - s3Client.moveObject(s3Bucket, s3DescriptorPath, s3ArchiveBucket, new S3Object(s3DescriptorPath), false); - } else { - log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); - s3Client.deleteObject(s3Bucket, s3DescriptorPath); - } + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); } } catch (ServiceException e) { diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java deleted file mode 100644 index bd169b05f2b..00000000000 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKillerConfig.java +++ /dev/null @@ -1,22 +0,0 @@ -package io.druid.storage.s3; - -import com.fasterxml.jackson.annotation.JsonProperty; - -public class S3DataSegmentKillerConfig -{ - @JsonProperty - public boolean archive = true; - - @JsonProperty - public String archiveBucket = ""; - - public boolean isArchive() - { - return archive; - } - - public String getArchiveBucket() - { - return archiveBucket; - } -} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java new file mode 100644 index 00000000000..245cbab8a63 --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -0,0 +1,99 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.storage.s3; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import io.druid.segment.loading.DataSegmentMover; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Object; + +import java.util.Map; + +public class S3DataSegmentMover implements DataSegmentMover +{ + private static final Logger log = new Logger(S3DataSegmentKiller.class); + + private final RestS3Service s3Client; + private final S3DataSegmentMoverConfig config; + + @Inject + public S3DataSegmentMover( + RestS3Service s3Client, + S3DataSegmentMoverConfig config + ) + { + this.s3Client = s3Client; + this.config = config; + } + + @Override + public DataSegment move(DataSegment segment) throws SegmentLoadingException + { + try { + Map loadSpec = segment.getLoadSpec(); + String s3Bucket = MapUtils.getString(loadSpec, "bucket"); + String s3Path = MapUtils.getString(loadSpec, "key"); + String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + + final String s3ArchiveBucket = config.getArchiveBucket(); + + if (s3ArchiveBucket.isEmpty()) { + log.warn("S3 archive bucket not specified, refusing to move segment [s3://%s/%s]", s3Bucket, s3Path); + return segment; + } + + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info( + "Moving index file[s3://%s/%s] to [s3://%s/%s]", + s3Bucket, + s3Path, + s3ArchiveBucket, + s3Path + ); + s3Client.moveObject(s3Bucket, s3Path, s3ArchiveBucket, new S3Object(s3Path), false); + } + if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + log.info( + "Moving descriptor file[s3://%s/%s] to [s3://%s/%s]", + s3Bucket, + s3DescriptorPath, + s3ArchiveBucket, + s3DescriptorPath + ); + s3Client.moveObject(s3Bucket, s3DescriptorPath, s3ArchiveBucket, new S3Object(s3DescriptorPath), false); + } + + return segment.withLoadSpec( + ImmutableMap.builder() + .putAll(loadSpec) + .put("bucket", s3ArchiveBucket).build() + ); + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "Unable to move segment[%s]", segment.getIdentifier()); + } + } +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java new file mode 100644 index 00000000000..a298ba9da39 --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.storage.s3; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class S3DataSegmentMoverConfig +{ + @JsonProperty + public String archiveBucket = ""; + + public String getArchiveBucket() + { + return archiveBucket; + } +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 6bb4624a3d3..fadba584bce 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -51,9 +51,10 @@ public class S3StorageDruidModule implements DruidModule Binders.dataSegmentPullerBinder(binder).addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); Binders.dataSegmentKillerBinder(binder).addBinding("s3_zip").to(S3DataSegmentKiller.class).in(LazySingleton.class); + Binders.dataSegmentMoverBinder(binder).addBinding("s3_zip").to(S3DataSegmentMover.class).in(LazySingleton.class); Binders.dataSegmentPusherBinder(binder).addBinding("s3").to(S3DataSegmentPusher.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentPusherConfig.class); - JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentKillerConfig.class); + JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentMoverConfig.class); Binders.taskLogsBinder(binder).addBinding("s3").to(S3TaskLogs.class); JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); diff --git a/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java new file mode 100644 index 00000000000..490c936011d --- /dev/null +++ b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java @@ -0,0 +1,57 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.loading; + +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import io.druid.timeline.DataSegment; + +import java.util.Map; + +public class OmniDataSegmentMover implements DataSegmentMover +{ + private final Map movers; + + @Inject + public OmniDataSegmentMover( + Map movers + ) + { + this.movers = movers; + } + + @Override + public DataSegment move(DataSegment segment) throws SegmentLoadingException + { + return getMover(segment).move(segment); + } + + private DataSegmentMover getMover(DataSegment segment) throws SegmentLoadingException + { + String type = MapUtils.getString(segment.getLoadSpec(), "type"); + DataSegmentMover mover = movers.get(type); + + if (mover == null) { + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, movers.keySet()); + } + + return mover; + } +} diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index e8a5b985bd4..450c9286554 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -61,7 +61,9 @@ import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; import io.druid.query.QuerySegmentWalker; import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.OmniDataSegmentKiller; +import io.druid.segment.loading.OmniDataSegmentMover; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.StorageLocationConfig; import io.druid.server.QueryResource; @@ -129,6 +131,8 @@ public class CliPeon extends GuiceRunnable // Build it to make it bind even if nothing binds to it. Binders.dataSegmentKillerBinder(binder); binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); + Binders.dataSegmentMoverBinder(binder); + binder.bind(DataSegmentMover.class).to(OmniDataSegmentMover.class).in(LazySingleton.class); binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class); binder.bind(ExecutorLifecycleConfig.class).toInstance( From cd7a941f83752bc13e17adddaab88dcffc057b75 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 9 Dec 2013 14:22:39 -0800 Subject: [PATCH 04/38] consolidate path functions --- .../java/io/druid/storage/s3/S3DataSegmentKiller.java | 3 +-- .../java/io/druid/storage/s3/S3DataSegmentMover.java | 2 +- .../java/io/druid/storage/s3/S3DataSegmentPusher.java | 9 ++++++--- .../src/main/java/io/druid/storage/s3/S3Utils.java | 4 ++++ 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java index 4e295d17aa4..0e4fde44d76 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java @@ -27,7 +27,6 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Object; import java.util.Map; @@ -54,7 +53,7 @@ public class S3DataSegmentKiller implements DataSegmentKiller Map loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); - String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 245cbab8a63..599e3dce463 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -56,7 +56,7 @@ public class S3DataSegmentMover implements DataSegmentMover Map loadSpec = segment.getLoadSpec(); String s3Bucket = MapUtils.getString(loadSpec, "bucket"); String s3Path = MapUtils.getString(loadSpec, "key"); - String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); final String s3ArchiveBucket = config.getArchiveBucket(); diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index e8b1a99710f..a73ed4d42ac 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -90,8 +90,11 @@ public class S3DataSegmentPusher implements DataSegmentPusher S3Object toPush = new S3Object(zipOutFile); final String outputBucket = config.getBucket(); + final String s3Path = outputKey + "/index.zip"; + final String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); + toPush.setBucketName(outputBucket); - toPush.setKey(outputKey + "/index.zip"); + toPush.setKey(s3Path); if (!config.getDisableAcl()) { toPush.setAcl(AccessControlList.REST_CANNED_AUTHENTICATED_READ); } @@ -116,7 +119,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(inSegment)), descriptorFile); S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); - descriptorObject.setKey(outputKey + "/descriptor.json"); + descriptorObject.setKey(s3DescriptorPath); if (!config.getDisableAcl()) { descriptorObject.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); } @@ -142,4 +145,4 @@ public class S3DataSegmentPusher implements DataSegmentPusher throw Throwables.propagate(e); } } -} \ No newline at end of file +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java index 3ae7088d88f..a4764717c1d 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java @@ -96,4 +96,8 @@ public class S3Utils return true; } + public static String descriptorPathForSegmentPath(String s3Path) + { + return s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + } } From bb1b037f876f7419db2c646bbc01e5440818a4fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 9 Dec 2013 17:42:00 -0800 Subject: [PATCH 05/38] fix docs --- docs/content/Configuration.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index e5c64c74c17..13a1a67a741 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -287,8 +287,7 @@ This deep storage is used to interface with Amazon's S3. |`druid.storage.bucket`|S3 bucket name.|none| |`druid.storage.basekey`|S3 base key.|none| |`druid.storage.disableAcl`|Boolean flag for ACL.|false| -|`druid.storage.archive`|Boolean flag. Archives killed segments instead of deleting them from S3|true| -|`druid.storage.archiveBucket`|S3 bucket name to archive segments to|none| +|`druid.storage.archiveBucket`|S3 bucket name where segments get archived to when running the indexing service *archive task*|none| #### HDFS Deep Storage From 4a291fdf3094886e20d65b87554a6a5debd0a4f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 11 Dec 2013 17:25:17 -0800 Subject: [PATCH 06/38] better naming --- .../io/druid/indexing/common/actions/SegmentMoveAction.java | 2 +- .../io/druid/indexing/overlord/IndexerDBCoordinator.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java index 67db4fc79be..c427709c5b3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java @@ -46,7 +46,7 @@ public class SegmentMoveAction implements TaskAction throw new ISE("Segments not covered by locks for task: %s", task.getId()); } - toolbox.getIndexerDBCoordinator().moveSegments(segments); + toolbox.getIndexerDBCoordinator().updateSegmentMetadata(segments); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java index 943e63fa821..f9db89b3fc9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java @@ -211,7 +211,7 @@ public class IndexerDBCoordinator return true; } - public void moveSegments(final Set segments) throws IOException + public void updateSegmentMetadata(final Set segments) throws IOException { dbi.inTransaction( new TransactionCallback() @@ -220,7 +220,7 @@ public class IndexerDBCoordinator public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception { for(final DataSegment segment : segments) { - moveSegment(handle, segment); + updatePayload(handle, segment); } return null; @@ -256,7 +256,7 @@ public class IndexerDBCoordinator .execute(); } - private void moveSegment(final Handle handle, final DataSegment segment) throws IOException + private void updatePayload(final Handle handle, final DataSegment segment) throws IOException { try { handle.createStatement( From 3ae48a8191bda71e900bee767847ce56873a8c57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 11 Dec 2013 17:25:29 -0800 Subject: [PATCH 07/38] fix cut-n-paste typo --- .../src/main/java/io/druid/storage/s3/S3DataSegmentMover.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 599e3dce463..96a5b8a8d9e 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -34,7 +34,7 @@ import java.util.Map; public class S3DataSegmentMover implements DataSegmentMover { - private static final Logger log = new Logger(S3DataSegmentKiller.class); + private static final Logger log = new Logger(S3DataSegmentMover.class); private final RestS3Service s3Client; private final S3DataSegmentMoverConfig config; From 3af6e49cd43bd6e2231739739dfbc18d43d1bcf5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 11 Dec 2013 17:29:04 -0800 Subject: [PATCH 08/38] throw exception instead of just printing a warning --- .../src/main/java/io/druid/storage/s3/S3DataSegmentMover.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 96a5b8a8d9e..9c79a079801 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -61,8 +61,7 @@ public class S3DataSegmentMover implements DataSegmentMover final String s3ArchiveBucket = config.getArchiveBucket(); if (s3ArchiveBucket.isEmpty()) { - log.warn("S3 archive bucket not specified, refusing to move segment [s3://%s/%s]", s3Bucket, s3Path); - return segment; + throw new SegmentLoadingException("S3 archive bucket not specified"); } if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { From 123bddd6151339c6cc38e05822c221071cff8c62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Dec 2013 14:29:09 -0800 Subject: [PATCH 09/38] update for new interfaces --- .../main/java/io/druid/indexing/common/task/MoveTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index 4ca600f8ea5..3154d42075d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -36,7 +36,7 @@ import org.joda.time.Interval; import java.util.List; -public class MoveTask extends AbstractTask +public class MoveTask extends AbstractFixedIntervalTask { private static final Logger log = new Logger(MoveTask.class); @@ -70,8 +70,8 @@ public class MoveTask extends AbstractTask throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); } - if(!myLock.getInterval().equals(getImplicitLockInterval().get())) { - throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getImplicitLockInterval().get()); + if(!myLock.getInterval().equals(getInterval())) { + throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval()); } // List unused segments From 6b903720021683c7d12b5befee9656fc4edea7ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Dec 2013 16:40:57 -0800 Subject: [PATCH 10/38] separate segment mover and segment archiver --- pom.xml | 2 +- .../storage/s3/S3DataSegmentArchiver.java | 59 +++++++++++++++++++ ....java => S3DataSegmentArchiverConfig.java} | 2 +- .../druid/storage/s3/S3DataSegmentMover.java | 34 ++++++----- .../storage/s3/S3StorageDruidModule.java | 3 +- 5 files changed, 82 insertions(+), 18 deletions(-) create mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java rename s3-extensions/src/main/java/io/druid/storage/s3/{S3DataSegmentMoverConfig.java => S3DataSegmentArchiverConfig.java} (96%) diff --git a/pom.xml b/pom.xml index f970d49dea5..f55d5202e2d 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.25.1 2.1.0-incubating - 0.1.6 + 0.1.7 diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java new file mode 100644 index 00000000000..b339187ff29 --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java @@ -0,0 +1,59 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.storage.s3; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import io.druid.segment.loading.DataSegmentArchiver; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + + +public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSegmentArchiver +{ + private final S3DataSegmentArchiverConfig config; + + @Inject + public S3DataSegmentArchiver( + RestS3Service s3Client, + S3DataSegmentArchiverConfig config + ) + { + super(s3Client); + this.config = config; + } + + @Override + public DataSegment archive(DataSegment segment) throws SegmentLoadingException + { + String targetS3Bucket = config.getArchiveBucket(); + String targetS3Path = MapUtils.getString(segment.getLoadSpec(), "key"); + + return move( + segment, + ImmutableMap.of( + "bucket", targetS3Bucket, + "key", targetS3Path + ) + ); + } +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java similarity index 96% rename from s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java index a298ba9da39..53a04e43107 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMoverConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java @@ -21,7 +21,7 @@ package io.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonProperty; -public class S3DataSegmentMoverConfig +public class S3DataSegmentArchiverConfig { @JsonProperty public String archiveBucket = ""; diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 9c79a079801..9b3f122b590 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -37,20 +37,17 @@ public class S3DataSegmentMover implements DataSegmentMover private static final Logger log = new Logger(S3DataSegmentMover.class); private final RestS3Service s3Client; - private final S3DataSegmentMoverConfig config; @Inject public S3DataSegmentMover( - RestS3Service s3Client, - S3DataSegmentMoverConfig config + RestS3Service s3Client ) { this.s3Client = s3Client; - this.config = config; } @Override - public DataSegment move(DataSegment segment) throws SegmentLoadingException + public DataSegment move(DataSegment segment, Map targetLoadSpec) throws SegmentLoadingException { try { Map loadSpec = segment.getLoadSpec(); @@ -58,10 +55,15 @@ public class S3DataSegmentMover implements DataSegmentMover String s3Path = MapUtils.getString(loadSpec, "key"); String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); - final String s3ArchiveBucket = config.getArchiveBucket(); + final String targetS3Bucket = MapUtils.getString(targetLoadSpec, "bucket"); + final String targetS3Path = MapUtils.getString(targetLoadSpec, "key"); + String targetS3DescriptorPath = S3Utils.descriptorPathForSegmentPath(targetS3Path); - if (s3ArchiveBucket.isEmpty()) { - throw new SegmentLoadingException("S3 archive bucket not specified"); + if (targetS3Bucket.isEmpty()) { + throw new SegmentLoadingException("Target S3 bucket is not specified"); + } + if (targetS3Path.isEmpty()) { + throw new SegmentLoadingException("Target S3 path is not specified"); } if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { @@ -69,26 +71,28 @@ public class S3DataSegmentMover implements DataSegmentMover "Moving index file[s3://%s/%s] to [s3://%s/%s]", s3Bucket, s3Path, - s3ArchiveBucket, - s3Path + targetS3Bucket, + targetS3Path ); - s3Client.moveObject(s3Bucket, s3Path, s3ArchiveBucket, new S3Object(s3Path), false); + s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, new S3Object(targetS3Path), false); } if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { log.info( "Moving descriptor file[s3://%s/%s] to [s3://%s/%s]", s3Bucket, s3DescriptorPath, - s3ArchiveBucket, - s3DescriptorPath + targetS3Bucket, + targetS3DescriptorPath ); - s3Client.moveObject(s3Bucket, s3DescriptorPath, s3ArchiveBucket, new S3Object(s3DescriptorPath), false); + s3Client.moveObject(s3Bucket, s3DescriptorPath, targetS3Bucket, new S3Object(targetS3DescriptorPath), false); } return segment.withLoadSpec( ImmutableMap.builder() .putAll(loadSpec) - .put("bucket", s3ArchiveBucket).build() + .put("bucket", targetS3Bucket) + .put("key", targetS3Path) + .build() ); } catch (ServiceException e) { diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index fadba584bce..d30f49f976a 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -52,9 +52,10 @@ public class S3StorageDruidModule implements DruidModule Binders.dataSegmentPullerBinder(binder).addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); Binders.dataSegmentKillerBinder(binder).addBinding("s3_zip").to(S3DataSegmentKiller.class).in(LazySingleton.class); Binders.dataSegmentMoverBinder(binder).addBinding("s3_zip").to(S3DataSegmentMover.class).in(LazySingleton.class); + Binders.dataSegmentArchiverBinder(binder).addBinding("s3_zip").to(S3DataSegmentArchiver.class).in(LazySingleton.class); Binders.dataSegmentPusherBinder(binder).addBinding("s3").to(S3DataSegmentPusher.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentPusherConfig.class); - JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentMoverConfig.class); + JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentArchiverConfig.class); Binders.taskLogsBinder(binder).addBinding("s3").to(S3TaskLogs.class); JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); From ac2ca0e46cb0624ae5c010b08f89facf6e42d4e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Dec 2013 17:55:31 -0800 Subject: [PATCH 11/38] separate move and archive tasks --- .../io/druid/indexing/common/TaskToolbox.java | 9 ++ .../indexing/common/TaskToolboxFactory.java | 5 + .../indexing/common/task/ArchiveTask.java | 106 ++++++++++++++++++ .../druid/indexing/common/task/MoveTask.java | 9 +- .../io/druid/indexing/common/task/Task.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 12 +- .../worker/WorkerTaskMonitorTest.java | 2 +- .../storage/s3/S3DataSegmentArchiver.java | 4 +- .../s3/S3DataSegmentArchiverConfig.java | 8 ++ .../druid/storage/s3/S3DataSegmentMover.java | 7 +- .../druid/storage/s3/S3DataSegmentPusher.java | 9 +- .../java/io/druid/storage/s3/S3Utils.java | 14 +++ .../loading/OmniDataSegmentArchiver.java | 57 ++++++++++ .../segment/loading/OmniDataSegmentMover.java | 4 +- .../src/main/java/io/druid/cli/CliPeon.java | 4 + 15 files changed, 233 insertions(+), 18 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java create mode 100644 server/src/main/java/io/druid/segment/loading/OmniDataSegmentArchiver.java 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 56f08f3c9f3..eb34f4e4c1b 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 @@ -29,6 +29,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPusher; @@ -53,6 +54,7 @@ public class TaskToolbox private final ServiceEmitter emitter; private final DataSegmentPusher segmentPusher; private final DataSegmentKiller dataSegmentKiller; + private final DataSegmentArchiver dataSegmentArchiver; private final DataSegmentMover dataSegmentMover; private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; @@ -71,6 +73,7 @@ public class TaskToolbox DataSegmentPusher segmentPusher, DataSegmentKiller dataSegmentKiller, DataSegmentMover dataSegmentMover, + DataSegmentArchiver dataSegmentArchiver, DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, @@ -88,6 +91,7 @@ public class TaskToolbox this.segmentPusher = segmentPusher; this.dataSegmentKiller = dataSegmentKiller; this.dataSegmentMover = dataSegmentMover; + this.dataSegmentArchiver = dataSegmentArchiver; this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; @@ -128,6 +132,11 @@ public class TaskToolbox return dataSegmentMover; } + public DataSegmentArchiver getDataSegmentArchiver() + { + return dataSegmentArchiver; + } + public DataSegmentAnnouncer getSegmentAnnouncer() { return segmentAnnouncer; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index cf9614c4bd6..d655edc34f0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -29,6 +29,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPusher; @@ -48,6 +49,7 @@ public class TaskToolboxFactory private final DataSegmentPusher segmentPusher; private final DataSegmentKiller dataSegmentKiller; private final DataSegmentMover dataSegmentMover; + private final DataSegmentArchiver dataSegmentArchiver; private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; @@ -64,6 +66,7 @@ public class TaskToolboxFactory DataSegmentPusher segmentPusher, DataSegmentKiller dataSegmentKiller, DataSegmentMover dataSegmentMover, + DataSegmentArchiver dataSegmentArchiver, DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, @@ -79,6 +82,7 @@ public class TaskToolboxFactory this.segmentPusher = segmentPusher; this.dataSegmentKiller = dataSegmentKiller; this.dataSegmentMover = dataSegmentMover; + this.dataSegmentArchiver = dataSegmentArchiver; this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; @@ -100,6 +104,7 @@ public class TaskToolboxFactory segmentPusher, dataSegmentKiller, dataSegmentMover, + dataSegmentArchiver, segmentAnnouncer, newSegmentServerView, queryRunnerFactoryConglomerate, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java new file mode 100644 index 00000000000..43f8fd60e18 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java @@ -0,0 +1,106 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentListUnusedAction; +import io.druid.indexing.common.actions.SegmentMoveAction; +import io.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.List; + +public class ArchiveTask extends AbstractFixedIntervalTask +{ + private static final Logger log = new Logger(ArchiveTask.class); + + public ArchiveTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval + ) + { + super(id, dataSource, interval); + } + + @Override + public String getType() + { + return "archive"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + // Confirm we have a lock (will throw if there isn't exactly one element) + final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox)); + + if (!myLock.getDataSource().equals(getDataSource())) { + throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); + } + + if (!myLock.getInterval().equals(getInterval())) { + throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval()); + } + + // List unused segments + final List unusedSegments = toolbox + .getTaskActionClient() + .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); + + // Verify none of these segments have versions > lock version + for (final DataSegment unusedSegment : unusedSegments) { + if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { + throw new ISE( + "WTF?! Unused segment[%s] has version[%s] > task version[%s]", + unusedSegment.getIdentifier(), + unusedSegment.getVersion(), + myLock.getVersion() + ); + } + + log.info("OK to archive segment: %s", unusedSegment.getIdentifier()); + } + + List archivedSegments = Lists.newLinkedList(); + + // Move segments + for (DataSegment segment : unusedSegments) { + archivedSegments.add(toolbox.getDataSegmentArchiver().archive(segment)); + } + + // Update metadata for moved segments + toolbox.getTaskActionClient().submit( + new SegmentMoveAction( + ImmutableSet.copyOf(archivedSegments) + ) + ); + + return TaskStatus.success(getId()); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index 3154d42075d..c8742f31c34 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -35,16 +35,20 @@ import io.druid.timeline.DataSegment; import org.joda.time.Interval; import java.util.List; +import java.util.Map; public class MoveTask extends AbstractFixedIntervalTask { private static final Logger log = new Logger(MoveTask.class); + private final Map targetLoadSpec; + @JsonCreator public MoveTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval + @JsonProperty("interval") Interval interval, + @JsonProperty("target") Map targetLoadSpec ) { super( @@ -52,6 +56,7 @@ public class MoveTask extends AbstractFixedIntervalTask dataSource, interval ); + this.targetLoadSpec = targetLoadSpec; } @Override @@ -97,7 +102,7 @@ public class MoveTask extends AbstractFixedIntervalTask // Move segments for (DataSegment segment : unusedSegments) { - movedSegments.add(toolbox.getDataSegmentMover().move(segment)); + movedSegments.add(toolbox.getDataSegmentMover().move(segment, targetLoadSpec)); } // Update metadata for moved segments diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 32747bdc7d3..8fa4b53bf10 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -46,6 +46,7 @@ import io.druid.query.QueryRunner; @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "move", value = MoveTask.class), + @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), 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 fcbfed7589b..a873daa876f 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 @@ -64,6 +64,7 @@ import io.druid.indexing.overlord.config.TaskQueueConfig; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPuller; @@ -89,6 +90,7 @@ import java.io.File; import java.io.IOException; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; public class TaskLifecycleTest @@ -162,11 +164,19 @@ public class TaskLifecycleTest new DataSegmentMover() { @Override - public DataSegment move(DataSegment dataSegment) throws SegmentLoadingException + public DataSegment move(DataSegment dataSegment, Map targetLoadSpec) throws SegmentLoadingException { return dataSegment; } }, + new DataSegmentArchiver() + { + @Override + public DataSegment archive(DataSegment segment) throws SegmentLoadingException + { + return segment; + } + }, null, // segment announcer null, // new segment server view null, // query runner factory conglomerate corporation unionized collective diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 7723013d239..8d4bf32b870 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -122,7 +122,7 @@ public class WorkerTaskMonitorTest new ThreadPoolTaskRunner( new TaskToolboxFactory( new TaskConfig(tmp.toString(), null, null, 0), - null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory( + null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory( new OmniSegmentLoader( ImmutableMap.of( "local", diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java index b339187ff29..b7c04ec0c00 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java @@ -46,13 +46,13 @@ public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSeg public DataSegment archive(DataSegment segment) throws SegmentLoadingException { String targetS3Bucket = config.getArchiveBucket(); - String targetS3Path = MapUtils.getString(segment.getLoadSpec(), "key"); + String targetS3BaseKey = config.getArchiveBaseKey(); return move( segment, ImmutableMap.of( "bucket", targetS3Bucket, - "key", targetS3Path + "baseKey", targetS3BaseKey ) ); } diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java index 53a04e43107..5eb33eb1b5d 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java @@ -26,8 +26,16 @@ public class S3DataSegmentArchiverConfig @JsonProperty public String archiveBucket = ""; + @JsonProperty + public String archiveBaseKey = ""; + public String getArchiveBucket() { return archiveBucket; } + + public String getArchiveBaseKey() + { + return archiveBaseKey; + } } diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 9b3f122b590..01558b5c44d 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; import io.druid.segment.loading.DataSegmentMover; +import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; @@ -56,14 +57,16 @@ public class S3DataSegmentMover implements DataSegmentMover String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); final String targetS3Bucket = MapUtils.getString(targetLoadSpec, "bucket"); - final String targetS3Path = MapUtils.getString(targetLoadSpec, "key"); + final String targetS3BaseKey = MapUtils.getString(targetLoadSpec, "baseKey"); + + final String targetS3Path = S3Utils.constructSegmentPath(targetS3BaseKey, segment); String targetS3DescriptorPath = S3Utils.descriptorPathForSegmentPath(targetS3Path); if (targetS3Bucket.isEmpty()) { throw new SegmentLoadingException("Target S3 bucket is not specified"); } if (targetS3Path.isEmpty()) { - throw new SegmentLoadingException("Target S3 path is not specified"); + throw new SegmentLoadingException("Target S3 baseKey is not specified"); } if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index a73ed4d42ac..664c270799b 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -20,7 +20,6 @@ package io.druid.storage.s3; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; @@ -29,7 +28,6 @@ import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; import io.druid.utils.CompressionUtils; import org.jets3t.service.ServiceException; @@ -45,7 +43,6 @@ import java.util.concurrent.Callable; public class S3DataSegmentPusher implements DataSegmentPusher { private static final EmittingLogger log = new EmittingLogger(S3DataSegmentPusher.class); - private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; private final S3DataSegmentPusherConfig config; @@ -73,10 +70,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher public DataSegment push(final File indexFilesDir, final DataSegment inSegment) throws IOException { log.info("Uploading [%s] to S3", indexFilesDir); - final String outputKey = JOINER.join( - config.getBaseKey().isEmpty() ? null : config.getBaseKey(), - DataSegmentPusherUtil.getStorageDir(inSegment) - ); + final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), inSegment); final File zipOutFile = File.createTempFile("druid", "index.zip"); final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); @@ -90,7 +84,6 @@ public class S3DataSegmentPusher implements DataSegmentPusher S3Object toPush = new S3Object(zipOutFile); final String outputBucket = config.getBucket(); - final String s3Path = outputKey + "/index.zip"; final String s3DescriptorPath = S3Utils.descriptorPathForSegmentPath(s3Path); toPush.setBucketName(outputBucket); diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java index a4764717c1d..6cf481fa2f9 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java @@ -19,9 +19,12 @@ package io.druid.storage.s3; +import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.metamx.common.RetryUtils; import org.jets3t.service.ServiceException; +import io.druid.segment.loading.DataSegmentPusherUtil; +import io.druid.timeline.DataSegment; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; @@ -34,6 +37,8 @@ import java.util.concurrent.Callable; */ public class S3Utils { + private static final Joiner JOINER = Joiner.on("/").skipNulls(); + public static void closeStreamsQuietly(S3Object s3Obj) { if (s3Obj == null) { @@ -96,6 +101,15 @@ public class S3Utils return true; } + + public static String constructSegmentPath(String baseKey, DataSegment segment) + { + return JOINER.join( + baseKey.isEmpty() ? null : baseKey, + DataSegmentPusherUtil.getStorageDir(segment) + ) + "/index.zip"; + } + public static String descriptorPathForSegmentPath(String s3Path) { return s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; diff --git a/server/src/main/java/io/druid/segment/loading/OmniDataSegmentArchiver.java b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentArchiver.java new file mode 100644 index 00000000000..bf34bbe17bb --- /dev/null +++ b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentArchiver.java @@ -0,0 +1,57 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.segment.loading; + +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import io.druid.timeline.DataSegment; + +import java.util.Map; + +public class OmniDataSegmentArchiver implements DataSegmentArchiver +{ + private final Map archivers; + + @Inject + public OmniDataSegmentArchiver( + Map archivers + ) + { + this.archivers = archivers; + } + + @Override + public DataSegment archive(DataSegment segment) throws SegmentLoadingException + { + return getArchiver(segment).archive(segment); + } + + private DataSegmentArchiver getArchiver(DataSegment segment) throws SegmentLoadingException + { + String type = MapUtils.getString(segment.getLoadSpec(), "type"); + DataSegmentArchiver archiver = archivers.get(type); + + if (archiver == null) { + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, archivers.keySet()); + } + + return archiver; + } +} diff --git a/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java index 490c936011d..d585b0b7db9 100644 --- a/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java +++ b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentMover.java @@ -38,9 +38,9 @@ public class OmniDataSegmentMover implements DataSegmentMover } @Override - public DataSegment move(DataSegment segment) throws SegmentLoadingException + public DataSegment move(DataSegment segment, Map targetLoadSpec) throws SegmentLoadingException { - return getMover(segment).move(segment); + return getMover(segment).move(segment, targetLoadSpec); } private DataSegmentMover getMover(DataSegment segment) throws SegmentLoadingException diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 450c9286554..7204e5fc63a 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -60,8 +60,10 @@ import io.druid.indexing.worker.executor.ChatHandlerResource; import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; import io.druid.query.QuerySegmentWalker; +import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; +import io.druid.segment.loading.OmniDataSegmentArchiver; import io.druid.segment.loading.OmniDataSegmentKiller; import io.druid.segment.loading.OmniDataSegmentMover; import io.druid.segment.loading.SegmentLoaderConfig; @@ -133,6 +135,8 @@ public class CliPeon extends GuiceRunnable binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); Binders.dataSegmentMoverBinder(binder); binder.bind(DataSegmentMover.class).to(OmniDataSegmentMover.class).in(LazySingleton.class); + Binders.dataSegmentArchiverBinder(binder); + binder.bind(DataSegmentArchiver.class).to(OmniDataSegmentArchiver.class).in(LazySingleton.class); binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class); binder.bind(ExecutorLifecycleConfig.class).toInstance( From e333776acac38eaf5419eaba12275b9b93b9f27d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 16 Dec 2013 13:22:07 -0800 Subject: [PATCH 12/38] rename SegmentMoveAction to SegmentMetadataUpdateAction --- ...mentMoveAction.java => SegmentMetadataUpdateAction.java} | 6 +++--- .../java/io/druid/indexing/common/actions/TaskAction.java | 2 +- .../java/io/druid/indexing/common/task/ArchiveTask.java | 4 ++-- .../main/java/io/druid/indexing/common/task/MoveTask.java | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) rename indexing-service/src/main/java/io/druid/indexing/common/actions/{SegmentMoveAction.java => SegmentMetadataUpdateAction.java} (92%) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java similarity index 92% rename from indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java rename to indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index c427709c5b3..f996a2c6ab0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMoveAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -13,13 +13,13 @@ import io.druid.timeline.DataSegment; import java.io.IOException; import java.util.Set; -public class SegmentMoveAction implements TaskAction +public class SegmentMetadataUpdateAction implements TaskAction { @JsonIgnore private final Set segments; @JsonCreator - public SegmentMoveAction( + public SegmentMetadataUpdateAction( @JsonProperty("segments") Set segments ) { @@ -70,7 +70,7 @@ public class SegmentMoveAction implements TaskAction @Override public String toString() { - return "SegmentMoveAction{" + + return "SegmentMetadataUpdateAction{" + "segments=" + segments + '}'; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index 1d59e40c6c3..038d06be3c6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -36,7 +36,7 @@ import java.io.IOException; @JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class), @JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class), @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), - @JsonSubTypes.Type(name = "segmentMove", value = SegmentMoveAction.class) + @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class) }) public interface TaskAction { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java index 43f8fd60e18..97747e211ab 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java @@ -29,7 +29,7 @@ import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUnusedAction; -import io.druid.indexing.common.actions.SegmentMoveAction; +import io.druid.indexing.common.actions.SegmentMetadataUpdateAction; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -96,7 +96,7 @@ public class ArchiveTask extends AbstractFixedIntervalTask // Update metadata for moved segments toolbox.getTaskActionClient().submit( - new SegmentMoveAction( + new SegmentMetadataUpdateAction( ImmutableSet.copyOf(archivedSegments) ) ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index c8742f31c34..371f0dc38a4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -30,7 +30,7 @@ import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUnusedAction; -import io.druid.indexing.common.actions.SegmentMoveAction; +import io.druid.indexing.common.actions.SegmentMetadataUpdateAction; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -106,7 +106,7 @@ public class MoveTask extends AbstractFixedIntervalTask } // Update metadata for moved segments - toolbox.getTaskActionClient().submit(new SegmentMoveAction( + toolbox.getTaskActionClient().submit(new SegmentMetadataUpdateAction( ImmutableSet.copyOf(movedSegments) )); From f3b8d9c047fc6a6dc6625119d1bbd088279136d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 16 Dec 2013 13:54:30 -0800 Subject: [PATCH 13/38] safely move files --- .../druid/storage/s3/S3DataSegmentMover.java | 53 ++++++++++++------- 1 file changed, 33 insertions(+), 20 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 01558b5c44d..fe8251a0cc0 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -69,26 +69,8 @@ public class S3DataSegmentMover implements DataSegmentMover throw new SegmentLoadingException("Target S3 baseKey is not specified"); } - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - log.info( - "Moving index file[s3://%s/%s] to [s3://%s/%s]", - s3Bucket, - s3Path, - targetS3Bucket, - targetS3Path - ); - s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, new S3Object(targetS3Path), false); - } - if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - log.info( - "Moving descriptor file[s3://%s/%s] to [s3://%s/%s]", - s3Bucket, - s3DescriptorPath, - targetS3Bucket, - targetS3DescriptorPath - ); - s3Client.moveObject(s3Bucket, s3DescriptorPath, targetS3Bucket, new S3Object(targetS3DescriptorPath), false); - } + safeMove(s3Bucket, s3Path, targetS3Bucket, targetS3Path); + safeMove(s3Bucket, s3DescriptorPath, targetS3Bucket, targetS3DescriptorPath); return segment.withLoadSpec( ImmutableMap.builder() @@ -102,4 +84,35 @@ public class S3DataSegmentMover implements DataSegmentMover throw new SegmentLoadingException(e, "Unable to move segment[%s]", segment.getIdentifier()); } } + + private void safeMove(String s3Bucket, String s3Path, String targetS3Bucket, String targetS3Path) + throws ServiceException, SegmentLoadingException + { + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info( + "Moving file[s3://%s/%s] to [s3://%s/%s]", + s3Bucket, + s3Path, + targetS3Bucket, + targetS3Path + ); + s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, new S3Object(targetS3Path), false); + } else { + // ensure object exists in target location + if(s3Client.isObjectInBucket(targetS3Bucket, targetS3Path)) { + log.info( + "Not moving file [s3://%s/%s], already present in target location [s3://%s/%s]", + s3Bucket, s3Path, + targetS3Bucket, targetS3Path + ); + } + else { + throw new SegmentLoadingException( + "Unable to move file [s3://%s/%s] to [s3://%s/%s], not present in either source or target location", + s3Bucket, s3Path, + targetS3Bucket, targetS3Path + ); + } + } + } } From f7f5ffc88084e7ae015bf23808c9ffade2a436c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 16 Dec 2013 15:12:15 -0800 Subject: [PATCH 14/38] add S3 segment move test and fix bug caught by test --- .../druid/storage/s3/S3DataSegmentMover.java | 16 +- .../storage/s3/S3DataSegmentMoverTest.java | 161 ++++++++++++++++++ 2 files changed, 175 insertions(+), 2 deletions(-) create mode 100644 s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index fe8251a0cc0..a9baf1d40ab 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -19,12 +19,13 @@ package io.druid.storage.s3; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; import io.druid.segment.loading.DataSegmentMover; -import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; @@ -74,7 +75,18 @@ public class S3DataSegmentMover implements DataSegmentMover return segment.withLoadSpec( ImmutableMap.builder() - .putAll(loadSpec) + .putAll( + Maps.filterKeys( + loadSpec, new Predicate() + { + @Override + public boolean apply(String input) + { + return !(input.equals("bucket") || input.equals("key")); + } + } + ) + ) .put("bucket", targetS3Bucket) .put("key", targetS3Path) .build() diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java new file mode 100644 index 00000000000..6206da881a4 --- /dev/null +++ b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java @@ -0,0 +1,161 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.storage.s3; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.metamx.common.MapUtils; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Object; +import org.jets3t.service.model.StorageObject; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; +import java.util.Set; + +public class S3DataSegmentMoverTest +{ + private static final DataSegment sourceSegment = new DataSegment( + "test", + new Interval("2013-01-01/2013-01-02"), + "1", + ImmutableMap.of( + "key", + "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip", + "bucket", + "main" + ), + ImmutableList.of("dim1", "dim1"), + ImmutableList.of("metric1", "metric2"), + new NoneShardSpec(), + 0, + 1 + ); + + @Test + public void testMove() throws Exception + { + MockStorageService mockS3Client = new MockStorageService(); + S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client); + + mockS3Client.putObject("main", new S3Object("baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip")); + mockS3Client.putObject("main", new S3Object("baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json")); + + DataSegment movedSegment = mover.move( + sourceSegment, + ImmutableMap.of("baseKey", "targetBaseKey", "bucket", "archive") + ); + + Map targetLoadSpec = movedSegment.getLoadSpec(); + Assert.assertEquals("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip", MapUtils.getString(targetLoadSpec, "key")); + Assert.assertEquals("archive", MapUtils.getString(targetLoadSpec, "bucket")); + Assert.assertTrue(mockS3Client.didMove()); + } + + @Test + public void testMoveNoop() throws Exception + { + MockStorageService mockS3Client = new MockStorageService(); + S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client); + + mockS3Client.putObject("archive", new S3Object("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip")); + mockS3Client.putObject("archive", new S3Object("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/descriptor.json")); + + DataSegment movedSegment = mover.move( + sourceSegment, + ImmutableMap.of("baseKey", "targetBaseKey", "bucket", "archive") + ); + + Map targetLoadSpec = movedSegment.getLoadSpec(); + + Assert.assertEquals("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip", MapUtils.getString(targetLoadSpec, "key")); + Assert.assertEquals("archive", MapUtils.getString(targetLoadSpec, "bucket")); + Assert.assertFalse(mockS3Client.didMove()); + } + + @Test(expected = SegmentLoadingException.class) + public void testMoveException() throws Exception + { + MockStorageService mockS3Client = new MockStorageService(); + S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client); + + mover.move( + sourceSegment, + ImmutableMap.of("baseKey", "targetBaseKey", "bucket", "archive") + ); + } + + private class MockStorageService extends RestS3Service { + Map> storage = Maps.newHashMap(); + boolean moved = false; + + private MockStorageService() throws S3ServiceException + { + super(null); + } + + public boolean didMove() { + return moved; + } + + @Override + public boolean isObjectInBucket(String bucketName, String objectKey) throws ServiceException + { + Set objects = storage.get(bucketName); + return (objects != null && objects.contains(objectKey)); + } + + @Override + public Map moveObject( + String sourceBucketName, + String sourceObjectKey, + String destinationBucketName, + StorageObject destinationObject, + boolean replaceMetadata + ) throws ServiceException + { + moved = true; + if(isObjectInBucket(sourceBucketName, sourceObjectKey)) { + this.putObject(destinationBucketName, new S3Object(destinationObject.getKey())); + storage.get(sourceBucketName).remove(sourceObjectKey); + } + return null; + } + + @Override + public S3Object putObject(String bucketName, S3Object object) throws S3ServiceException + { + if (!storage.containsKey(bucketName)) { + storage.put(bucketName, Sets.newHashSet()); + } + storage.get(bucketName).add(object.getKey()); + return object; + } + } +} From 2c7a6d26b394abfdbd5cdfb6de70fb5e5d28bdf6 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Dec 2013 15:55:28 -0800 Subject: [PATCH 15/38] fix typo in tutorial --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index e3fe41c51c6..1d5cf883267 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -205,7 +205,7 @@ You are probably wondering, what are these [Granularities](Granularities.html) a To issue the query and get some results, run the following in your command line: ``` -curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json' -d ````timeseries_query.body +curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json' -d @timeseries_query.body ``` Once again, you should get a JSON blob of text back with your results, that looks something like this: From f2241c8885b75e4482c234bf45ae57f58b378254 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Dec 2013 16:16:00 -0800 Subject: [PATCH 16/38] prepare for next release --- build.sh | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Realtime.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.textile | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- services/src/main/java/io/druid/cli/CliCoordinator.java | 2 +- services/src/main/java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/CliHistorical.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliRealtime.java | 2 +- services/src/main/java/io/druid/cli/CliRealtimeExample.java | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/build.sh b/build.sh index e0e5f513761..b5f28547e86 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.36" +echo "See also http://druid.io/docs/0.6.37" diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index db17223c8d5..c671828fd62 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: doc_page --- # Booting a Single Node Cluster # -[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.36-bin.tar.gz). +[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.37-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/content/Examples.md b/docs/content/Examples.md index fd18b24f1b9..90813d047b0 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.36 +git checkout druid-0.6.37 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.36-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index 71990ad57d9..87192ed88a5 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.36"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.37"] druid.zk.service.host=localhost diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 1d5cf883267..391302dfd69 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.36-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.36 +cd druid-services-0.6.37 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index dc834ac5fd9..28448745e30 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -44,7 +44,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h #### Setting up Kafka -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.36/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.37/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 3cba7a77582..a48bf0ebc46 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.36-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.36"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.37"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -238,7 +238,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.36","io.druid.extensions:druid-kafka-seven:0.6.36"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.37","io.druid.extensions:druid-kafka-seven:0.6.37"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index bf201263c3a..ddc063ecb24 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.36-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.36 +cd druid-services-0.6.37 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.textile index 5a0ef90dcf7..9b7f902ca53 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.textile @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.36-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz. Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index ce569f88f26..bd66c20d095 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.36"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.37"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 9ba5d55acc4..37d6cbd53ea 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.36","io.druid.extensions:druid-kafka-seven:0.6.36","io.druid.extensions:druid-rabbitmq:0.6.36"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.37","io.druid.extensions:druid-kafka-seven:0.6.37","io.druid.extensions:druid-rabbitmq:0.6.37"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index b099dfa69cf..7768b8c54cb 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -53,7 +53,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.36/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/0.6.37/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 47240c691ee..a7ce240a011 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -60,7 +60,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.36/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.37/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index e873f26c70c..5890a714eaa 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,7 +41,7 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.36/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.37/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index ca5449100d5..e3638a19c83 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.36/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.37/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 2c258625e4b..44f5816fda8 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -95,7 +95,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.36/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/0.6.37/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index 065db03dba5..fe3d90f476a 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.36/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/0.6.37/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 819debc339a..83c9628ec49 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -42,7 +42,7 @@ import java.util.concurrent.Executor; */ @Command( name = "realtime", - description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.36/Realtime.html for a description" + description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.37/Realtime.html for a description" ) public class CliRealtimeExample extends ServerRunnable { From 7e0b4d4fd211b58a2a0d2fe507bcca0291158528 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Dec 2013 16:17:41 -0800 Subject: [PATCH 17/38] [maven-release-plugin] prepare release druid-0.6.37 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index d2c783a089d..d9b75f4ee47 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/common/pom.xml b/common/pom.xml index 943aec92a08..3698e93ee65 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/examples/pom.xml b/examples/pom.xml index c0019b4c9cc..9940be68bc9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 81247522e1d..1f5d6d129eb 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6a5e90cd0c6..c148c6cfda4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 07974bc9ff0..a9b39f7a79f 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index ebeba81d6d5..cc360d9fb13 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index c7b08ebc463..09830ac8097 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/pom.xml b/pom.xml index f55d5202e2d..d236f258da6 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.37-SNAPSHOT + 0.6.37 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.37 diff --git a/processing/pom.xml b/processing/pom.xml index 9cf11cd9a6d..e51f390e9a1 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 29031970b84..18dd990cd8b 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index a5b859c7ea7..022b5801ca4 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/server/pom.xml b/server/pom.xml index bf05da8aec9..ffc1a0ebf8e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 diff --git a/services/pom.xml b/services/pom.xml index 306803a9ac7..537f7c89d28 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.37-SNAPSHOT + 0.6.37 From 48d677d1356f0d3974fecaba86535c13b708f558 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Dec 2013 16:17:45 -0800 Subject: [PATCH 18/38] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index d9b75f4ee47..91c20c76440 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 3698e93ee65..81fe6424308 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9940be68bc9..ef97db4dd75 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 1f5d6d129eb..47b5c0f6d11 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index c148c6cfda4..64693a1ee9f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a9b39f7a79f..34e7e27d500 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index cc360d9fb13..30a863a29a1 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 09830ac8097..cf14732dafe 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/pom.xml b/pom.xml index d236f258da6..288cbecef8f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.37 + 0.6.38-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.37 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index e51f390e9a1..07e60d195f8 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 18dd990cd8b..60ac9636ae9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 022b5801ca4..7183ef439e4 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ffc1a0ebf8e..028b8b3837a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 537f7c89d28..1f20ba42268 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.37 + 0.6.38-SNAPSHOT From 178c26f9f7db704515e544910a0a123927796791 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 16 Dec 2013 16:50:31 -0800 Subject: [PATCH 19/38] fix naming and docs --- docs/content/Configuration.md | 5 +++-- .../java/io/druid/storage/s3/S3DataSegmentArchiver.java | 3 +-- .../io/druid/storage/s3/S3DataSegmentArchiverConfig.java | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 13a1a67a741..e317ac0ecd4 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -285,9 +285,10 @@ This deep storage is used to interface with Amazon's S3. |Property|Description|Default| |--------|-----------|-------| |`druid.storage.bucket`|S3 bucket name.|none| -|`druid.storage.basekey`|S3 base key.|none| +|`druid.storage.basekey`|S3 object key prefix for storage.|none| |`druid.storage.disableAcl`|Boolean flag for ACL.|false| -|`druid.storage.archiveBucket`|S3 bucket name where segments get archived to when running the indexing service *archive task*|none| +|`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none| +|`druid.storage.archiveBasekey`|S3 object key prefix for archiving.|none| #### HDFS Deep Storage diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java index b7c04ec0c00..1c642f110bd 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java @@ -21,7 +21,6 @@ package io.druid.storage.s3; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; -import com.metamx.common.MapUtils; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; @@ -46,7 +45,7 @@ public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSeg public DataSegment archive(DataSegment segment) throws SegmentLoadingException { String targetS3Bucket = config.getArchiveBucket(); - String targetS3BaseKey = config.getArchiveBaseKey(); + String targetS3BaseKey = config.getArchiveBasekey(); return move( segment, diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java index 5eb33eb1b5d..9aeccb74afe 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java @@ -27,15 +27,15 @@ public class S3DataSegmentArchiverConfig public String archiveBucket = ""; @JsonProperty - public String archiveBaseKey = ""; + public String archiveBasekey = ""; public String getArchiveBucket() { return archiveBucket; } - public String getArchiveBaseKey() + public String getArchiveBasekey() { - return archiveBaseKey; + return archiveBasekey; } } From 3b2833d55dcc429ae3547c2eb5f6bc05459f98f9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Dec 2013 21:05:14 -0800 Subject: [PATCH 20/38] JavaScriptAggregatorFactory: Handle missing columns by passing down null args --- .../JavaScriptAggregatorFactory.java | 8 +++-- .../aggregation/JavaScriptAggregatorTest.java | 34 +++++++++++++++++++ 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index ff55c9cee7f..927ab89676f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -265,9 +265,11 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory final Object[] args = new Object[size + 1]; args[0] = current; - int i = 0; - while (i < size) { - args[i + 1] = selectorList[i++].get(); + for (int i = 0 ; i < size ; i++) { + final ObjectColumnSelector selector = selectorList[i]; + if (selector != null) { + args[i + 1] = selector.get(); + } } final Object res = fnAggregate.call(cx, scope, scope, args); diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java index 2435211dfe9..7f087559339 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java @@ -27,6 +27,7 @@ import org.junit.Test; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Map; public class JavaScriptAggregatorTest @@ -141,6 +142,39 @@ public class JavaScriptAggregatorTest Assert.assertEquals(val, agg.get(buf, position)); } + @Test + public void testAggregateMissingColumn() + { + Map script = scriptDoubleSum; + + JavaScriptAggregator agg = new JavaScriptAggregator( + "billy", + Collections.singletonList(null), + JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"), + script.get("fnReset"), + script.get("fnCombine")) + ); + + final double val = 0; + + Assert.assertEquals("billy", agg.getName()); + + agg.reset(); + Assert.assertEquals(val, agg.get()); + Assert.assertEquals(val, agg.get()); + Assert.assertEquals(val, agg.get()); + + agg.aggregate(); + Assert.assertEquals(val, agg.get()); + Assert.assertEquals(val, agg.get()); + Assert.assertEquals(val, agg.get()); + + agg.aggregate(); + Assert.assertEquals(val, agg.get()); + Assert.assertEquals(val, agg.get()); + Assert.assertEquals(val, agg.get()); + } + public static void main(String... args) throws Exception { final LoopingFloatColumnSelector selector = new LoopingFloatColumnSelector(new float[]{42.12f, 9f}); From af202d7576f626561c22d50f6611e2885630f7f3 Mon Sep 17 00:00:00 2001 From: Hagen Rother Date: Tue, 17 Dec 2013 15:27:17 +0100 Subject: [PATCH 21/38] improve kafka intake stability --- .../firehose/kafka/KafkaEightFirehoseFactory.java | 10 +++++++++- .../firehose/kafka/KafkaSevenFirehoseFactory.java | 10 +++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java index c8bf876cc92..86c165c04d8 100644 --- a/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java +++ b/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java @@ -107,7 +107,15 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory return null; } - return parser.parse(ByteBuffer.wrap(message)); + try { + return parser.parse(ByteBuffer.wrap(message)); + } + catch (Exception e) { + throw new FormattedException.Builder() + .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) + .withMessage(String.format("Error parsing[%s], got [%s]", ByteBuffer.wrap(message), e.toString())) + .build(); + } } @Override diff --git a/kafka-seven/src/main/java/io/druid/firehose/kafka/KafkaSevenFirehoseFactory.java b/kafka-seven/src/main/java/io/druid/firehose/kafka/KafkaSevenFirehoseFactory.java index 8f200d1cdbc..c227b323877 100644 --- a/kafka-seven/src/main/java/io/druid/firehose/kafka/KafkaSevenFirehoseFactory.java +++ b/kafka-seven/src/main/java/io/druid/firehose/kafka/KafkaSevenFirehoseFactory.java @@ -120,7 +120,15 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory public InputRow parseMessage(Message message) throws FormattedException { - return parser.parse(message.payload()); + try { + return parser.parse(message.payload()); + } + catch (Exception e) { + throw new FormattedException.Builder() + .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) + .withMessage(String.format("Error parsing[%s], got [%s]", message.payload(), e.toString())) + .build(); + } } @Override From 58d1262edff4b513f824d31d6e3792ace2cf083e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Dec 2013 08:16:49 -0800 Subject: [PATCH 22/38] Indexing console: Clarify "Complete" with "recently completed" --- indexing-service/src/main/resources/indexer_static/console.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/resources/indexer_static/console.html b/indexing-service/src/main/resources/indexer_static/console.html index f51383c72c0..e8221aa287e 100644 --- a/indexing-service/src/main/resources/indexer_static/console.html +++ b/indexing-service/src/main/resources/indexer_static/console.html @@ -51,7 +51,7 @@
Loading Waiting Tasks... this may take a few minutes
-

Complete Tasks

+

Complete Tasks - Tasks recently completed

Loading Complete Tasks... this may take a few minutes
From bbb2754a7c3a36123da49b79c86ae5abdd3f8a4c Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 16:22:36 -0800 Subject: [PATCH 23/38] fix redirects in druid --- .../main/java/io/druid/cli/CliOverlord.java | 26 ++++++++--------- .../CoordinatorJettyServerInitializer.java | 29 +++++++++---------- 2 files changed, 26 insertions(+), 29 deletions(-) diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 44f5816fda8..9ae0fd99227 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -79,9 +79,7 @@ import io.druid.tasklogs.TaskLogStreamer; import io.druid.tasklogs.TaskLogs; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerList; -import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; @@ -238,12 +236,12 @@ public class CliOverlord extends ServerRunnable @Override public void initialize(Server server, Injector injector) { - final ServletContextHandler redirect = new ServletContextHandler(ServletContextHandler.SESSIONS); - redirect.setContextPath("/"); - redirect.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - final ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setBaseResource( + ServletHolder holderPwd = new ServletHolder("default", DefaultServlet.class); + + root.addServlet(holderPwd, "/"); + root.setBaseResource( new ResourceCollection( new String[]{ TaskMaster.class.getClassLoader().getResource("static").toExternalForm(), @@ -251,17 +249,17 @@ public class CliOverlord extends ServerRunnable } ) ); + //root.setResourceBase(DruidCoordinator.class.getClassLoader().getResource("static").toExternalForm()); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GzipFilter.class, "/*", null); - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.setContextPath("/"); + // Can't use /* here because of Guice and Jetty static content conflicts + root.addFilter(GuiceFilter.class, "/druid/*", null); HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{redirect, resourceHandler, root, new DefaultHandler()}); - server.setHandler(handlerList); + handlerList.setHandlers(new Handler[]{root}); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); - root.addFilter(GuiceFilter.class, "/*", null); + server.setHandler(handlerList); } } } diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 12cf906b3cc..2ba09f24717 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -26,9 +26,7 @@ import io.druid.server.http.RedirectFilter; import io.druid.server.initialization.JettyServerInitializer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerList; -import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; @@ -42,22 +40,23 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer @Override public void initialize(Server server, Injector injector) { - final ServletContextHandler redirect = new ServletContextHandler(ServletContextHandler.SESSIONS); - redirect.setContextPath("/"); - redirect.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); - - final ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setResourceBase(DruidCoordinator.class.getClassLoader().getResource("static").toExternalForm()); - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.setContextPath("/"); + + ServletHolder holderPwd = new ServletHolder("default", DefaultServlet.class); + + root.addServlet(holderPwd, "/"); + root.setResourceBase(DruidCoordinator.class.getClassLoader().getResource("static").toExternalForm()); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GzipFilter.class, "/*", null); + + // Can't use '/*' here because of Guice and Jetty static content conflicts + // The coordinator really needs a standarized api path + root.addFilter(GuiceFilter.class, "/info/*", null); + root.addFilter(GuiceFilter.class, "/coordinator/*", null); HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{redirect, resourceHandler, root, new DefaultHandler()}); - server.setHandler(handlerList); + handlerList.setHandlers(new Handler[]{root}); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); - root.addFilter(GuiceFilter.class, "/*", null); + server.setHandler(handlerList); } } From 69f33b0acd225cc1a31c6a80e0e42293ba26eb5d Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 16:24:15 -0800 Subject: [PATCH 24/38] remove commented line --- services/src/main/java/io/druid/cli/CliOverlord.java | 1 - 1 file changed, 1 deletion(-) diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 9ae0fd99227..6b6123655cf 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -249,7 +249,6 @@ public class CliOverlord extends ServerRunnable } ) ); - //root.setResourceBase(DruidCoordinator.class.getClassLoader().getResource("static").toExternalForm()); root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); root.addFilter(GzipFilter.class, "/*", null); From 07875dd30bfca7572f8b110499da0604c7d129d8 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 16:28:11 -0800 Subject: [PATCH 25/38] prepare for next release --- build.sh | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Realtime.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.textile | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- services/src/main/java/io/druid/cli/CliCoordinator.java | 2 +- services/src/main/java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/CliHistorical.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliRealtime.java | 2 +- services/src/main/java/io/druid/cli/CliRealtimeExample.java | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/build.sh b/build.sh index b5f28547e86..152b403ce2c 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.37" +echo "See also http://druid.io/docs/0.6.38" diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index c671828fd62..478e1f409f0 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: doc_page --- # Booting a Single Node Cluster # -[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.37-bin.tar.gz). +[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.38-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 90813d047b0..528f15876ad 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.37 +git checkout druid-0.6.38 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index 87192ed88a5..c604a1405c3 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.37"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.38"] druid.zk.service.host=localhost diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 391302dfd69..d20d4e0ec83 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.37 +cd druid-services-0.6.38 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index 28448745e30..cd6df42185c 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -44,7 +44,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h #### Setting up Kafka -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.37/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.38/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index a48bf0ebc46..c8c7c8e4b93 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.37"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.38"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -238,7 +238,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.37","io.druid.extensions:druid-kafka-seven:0.6.37"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.38","io.druid.extensions:druid-kafka-seven:0.6.38"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index ddc063ecb24..8d5e75a0da8 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.37 +cd druid-services-0.6.38 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.textile index 9b7f902ca53..ae36ab176b4 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.textile @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.37-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz. Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index bd66c20d095..ae40823c151 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.37"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.38"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 37d6cbd53ea..b92d608ed58 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.37","io.druid.extensions:druid-kafka-seven:0.6.37","io.druid.extensions:druid-rabbitmq:0.6.37"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.38","io.druid.extensions:druid-kafka-seven:0.6.38","io.druid.extensions:druid-rabbitmq:0.6.38"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 7768b8c54cb..c69eba26445 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -53,7 +53,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.37/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/0.6.38/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index a7ce240a011..de50d2c1042 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -60,7 +60,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.37/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.38/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index 5890a714eaa..4f4bdd1336a 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,7 +41,7 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.37/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.38/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index e3638a19c83..746eb88113a 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.37/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.38/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 6b6123655cf..077654a335f 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -93,7 +93,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.37/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/0.6.38/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index fe3d90f476a..d414cf673f4 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.37/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/0.6.38/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 83c9628ec49..66021e22c66 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -42,7 +42,7 @@ import java.util.concurrent.Executor; */ @Command( name = "realtime", - description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.37/Realtime.html for a description" + description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.38/Realtime.html for a description" ) public class CliRealtimeExample extends ServerRunnable { From ed9f4b4cf2f738657c9024a306d6f76d25ee3a4a Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 16:29:49 -0800 Subject: [PATCH 26/38] [maven-release-plugin] prepare release druid-0.6.38 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 91c20c76440..534eaab7b2f 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/common/pom.xml b/common/pom.xml index 81fe6424308..60f25e603d8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/examples/pom.xml b/examples/pom.xml index ef97db4dd75..3fdafea8faa 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 47b5c0f6d11..416b70827e8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 64693a1ee9f..bea37023db5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 34e7e27d500..b29a2e70f9a 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 30a863a29a1..813cae3a6e4 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index cf14732dafe..86374e5883e 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/pom.xml b/pom.xml index 288cbecef8f..539971c9099 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.38-SNAPSHOT + 0.6.38 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.38 diff --git a/processing/pom.xml b/processing/pom.xml index 07e60d195f8..efe264fab49 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 60ac9636ae9..8fa443b6c18 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 7183ef439e4..2aea865aecb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/server/pom.xml b/server/pom.xml index 028b8b3837a..222f63bf8b3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 diff --git a/services/pom.xml b/services/pom.xml index 1f20ba42268..9b36b8279ab 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.38-SNAPSHOT + 0.6.38 From 5932150d24ed8c565f9b4f9c4e13d89dd974fb48 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 16:29:53 -0800 Subject: [PATCH 27/38] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 534eaab7b2f..a558bc27751 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 60f25e603d8..c5e1a28fd2a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 3fdafea8faa..62ac9caa341 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 416b70827e8..cc85881c688 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index bea37023db5..9c1800a508f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index b29a2e70f9a..4c94d67e16d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 813cae3a6e4..dab6df1961c 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 86374e5883e..b118d103e36 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/pom.xml b/pom.xml index 539971c9099..526caaf9e8a 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.38 + 0.6.39-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.38 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index efe264fab49..f515f1f4289 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 8fa443b6c18..8cf219ea40e 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 2aea865aecb..92562336cfc 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 222f63bf8b3..c1c4ae6e806 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 9b36b8279ab..3b20b66d8b1 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.38 + 0.6.39-SNAPSHOT From 0051877f8486d06f05c46ed96400da7c89442d61 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 17:46:16 -0800 Subject: [PATCH 28/38] fix status endpoint --- services/src/main/java/io/druid/cli/CliOverlord.java | 1 + .../java/io/druid/cli/CoordinatorJettyServerInitializer.java | 1 + 2 files changed, 2 insertions(+) diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 077654a335f..feb29bccfcd 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -253,6 +253,7 @@ public class CliOverlord extends ServerRunnable root.addFilter(GzipFilter.class, "/*", null); // Can't use /* here because of Guice and Jetty static content conflicts + root.addFilter(GuiceFilter.class, "/status/*", null); root.addFilter(GuiceFilter.class, "/druid/*", null); HandlerList handlerList = new HandlerList(); diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 2ba09f24717..be7a59ea2a8 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -51,6 +51,7 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer // Can't use '/*' here because of Guice and Jetty static content conflicts // The coordinator really needs a standarized api path + root.addFilter(GuiceFilter.class, "/status/*", null); root.addFilter(GuiceFilter.class, "/info/*", null); root.addFilter(GuiceFilter.class, "/coordinator/*", null); From d112b2d2111288a75de303dbb50c29e9feab0eda Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 17:47:04 -0800 Subject: [PATCH 29/38] prepare for next release --- build.sh | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Realtime.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.textile | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- services/src/main/java/io/druid/cli/CliCoordinator.java | 2 +- services/src/main/java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/CliHistorical.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliRealtime.java | 2 +- services/src/main/java/io/druid/cli/CliRealtimeExample.java | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/build.sh b/build.sh index 152b403ce2c..f9bd7723587 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.38" +echo "See also http://druid.io/docs/0.6.39" diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 478e1f409f0..ded3b7128d9 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: doc_page --- # Booting a Single Node Cluster # -[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.38-bin.tar.gz). +[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.39-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 528f15876ad..00aa3c1d4ff 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.38 +git checkout druid-0.6.39 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index c604a1405c3..7ce371dfea4 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.38"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.39"] druid.zk.service.host=localhost diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index d20d4e0ec83..8b5bde905d8 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.38 +cd druid-services-0.6.39 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index cd6df42185c..ab4d91651ca 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -44,7 +44,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h #### Setting up Kafka -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.38/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.39/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index c8c7c8e4b93..6599031a0b2 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.38"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.39"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -238,7 +238,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.38","io.druid.extensions:druid-kafka-seven:0.6.38"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.39","io.druid.extensions:druid-kafka-seven:0.6.39"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 8d5e75a0da8..df7b4d2df24 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.38 +cd druid-services-0.6.39 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.textile index ae36ab176b4..6fab02906a1 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.textile @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.38-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz. Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index ae40823c151..785f3901eb4 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.38"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.39"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index b92d608ed58..ae8675c610f 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.38","io.druid.extensions:druid-kafka-seven:0.6.38","io.druid.extensions:druid-rabbitmq:0.6.38"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.39","io.druid.extensions:druid-kafka-seven:0.6.39","io.druid.extensions:druid-rabbitmq:0.6.39"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index c69eba26445..4b6d6b93721 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -53,7 +53,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.38/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/0.6.39/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index de50d2c1042..ea71bc7de58 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -60,7 +60,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.38/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.39/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index 4f4bdd1336a..c9331bd0f04 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,7 +41,7 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.38/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.39/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 746eb88113a..0072e5f3de5 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.38/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.39/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index feb29bccfcd..abb516803dc 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -93,7 +93,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.38/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/0.6.39/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index d414cf673f4..e5a68c647c0 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.38/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/0.6.39/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 66021e22c66..0ecb83e284a 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -42,7 +42,7 @@ import java.util.concurrent.Executor; */ @Command( name = "realtime", - description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.38/Realtime.html for a description" + description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.39/Realtime.html for a description" ) public class CliRealtimeExample extends ServerRunnable { From 2a98a4d3e0b6706c9553933d033145f495867162 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 17:48:42 -0800 Subject: [PATCH 30/38] [maven-release-plugin] prepare release druid-0.6.39 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index a558bc27751..eb4d75235fa 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/common/pom.xml b/common/pom.xml index c5e1a28fd2a..a1c3a8b7834 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/examples/pom.xml b/examples/pom.xml index 62ac9caa341..f941e500333 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index cc85881c688..e1506b975c1 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 9c1800a508f..29a33848ba9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4c94d67e16d..2703aa29fba 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index dab6df1961c..bd20e066210 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index b118d103e36..ed60b704a07 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/pom.xml b/pom.xml index 526caaf9e8a..64f1cf045c6 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.39-SNAPSHOT + 0.6.39 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.39 diff --git a/processing/pom.xml b/processing/pom.xml index f515f1f4289..ade2f8ecd28 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 8cf219ea40e..23935dc15af 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 92562336cfc..773fd1aff02 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/server/pom.xml b/server/pom.xml index c1c4ae6e806..5f335bab421 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 diff --git a/services/pom.xml b/services/pom.xml index 3b20b66d8b1..48a4e5bcbcc 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.39-SNAPSHOT + 0.6.39 From 494ec530501310f42c7f1105c5330531c266cfa0 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Dec 2013 17:48:46 -0800 Subject: [PATCH 31/38] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index eb4d75235fa..07074721737 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index a1c3a8b7834..0aa0c1221c6 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index f941e500333..0b4f6d94a58 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index e1506b975c1..c0d3015bc39 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 29a33848ba9..58b73469c86 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 2703aa29fba..f3890bc98ea 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index bd20e066210..2bb456703a6 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index ed60b704a07..b55cc2ad3a1 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/pom.xml b/pom.xml index 64f1cf045c6..0163a2e34fe 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.39 + 0.6.40-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.39 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index ade2f8ecd28..3baa76d67be 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 23935dc15af..f7264dd1a0a 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 773fd1aff02..c08f57ef0fc 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 5f335bab421..79fe38908ee 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 48a4e5bcbcc..d6dd6892869 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.39 + 0.6.40-SNAPSHOT From fd427a33288aadc1b307bdc3f582d2cd926e9772 Mon Sep 17 00:00:00 2001 From: Hagen Rother Date: Wed, 18 Dec 2013 17:22:28 +0100 Subject: [PATCH 32/38] bump the maven shader plugin and config --- services/pom.xml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/services/pom.xml b/services/pom.xml index d6dd6892869..fd57bbc366d 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -68,7 +68,7 @@ org.apache.maven.plugins maven-shade-plugin - 1.6 + 2.2 package @@ -89,6 +89,9 @@ + + + From 892a42bafe6550cb2069850ab7f6bb1daa0969bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 18 Dec 2013 14:37:27 -0800 Subject: [PATCH 33/38] fix docs config casing --- docs/content/Configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index e317ac0ecd4..f01b5d41a0b 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -285,10 +285,10 @@ This deep storage is used to interface with Amazon's S3. |Property|Description|Default| |--------|-----------|-------| |`druid.storage.bucket`|S3 bucket name.|none| -|`druid.storage.basekey`|S3 object key prefix for storage.|none| +|`druid.storage.baseKey`|S3 object key prefix for storage.|none| |`druid.storage.disableAcl`|Boolean flag for ACL.|false| |`druid.storage.archiveBucket`|S3 bucket name for archiving when running the indexing-service *archive task*.|none| -|`druid.storage.archiveBasekey`|S3 object key prefix for archiving.|none| +|`druid.storage.archiveBaseKey`|S3 object key prefix for archiving.|none| #### HDFS Deep Storage From c8be38fe40de5351f4c2e1a2be3e6b076fff94da Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 18 Dec 2013 14:57:14 -0800 Subject: [PATCH 34/38] back to baseKey --- .../java/io/druid/storage/s3/S3DataSegmentArchiver.java | 2 +- .../io/druid/storage/s3/S3DataSegmentArchiverConfig.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java index 1c642f110bd..0da038352f0 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiver.java @@ -45,7 +45,7 @@ public class S3DataSegmentArchiver extends S3DataSegmentMover implements DataSeg public DataSegment archive(DataSegment segment) throws SegmentLoadingException { String targetS3Bucket = config.getArchiveBucket(); - String targetS3BaseKey = config.getArchiveBasekey(); + String targetS3BaseKey = config.getArchiveBaseKey(); return move( segment, diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java index 9aeccb74afe..5eb33eb1b5d 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentArchiverConfig.java @@ -27,15 +27,15 @@ public class S3DataSegmentArchiverConfig public String archiveBucket = ""; @JsonProperty - public String archiveBasekey = ""; + public String archiveBaseKey = ""; public String getArchiveBucket() { return archiveBucket; } - public String getArchiveBasekey() + public String getArchiveBaseKey() { - return archiveBasekey; + return archiveBaseKey; } } From 1ff855d744782d0b1eab8a82cee2c2bacb0d2853 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Dec 2013 15:16:57 -0800 Subject: [PATCH 35/38] Fix MoveTask serde and ArchiveTask id creation --- .../indexing/common/task/ArchiveTask.java | 6 ++- .../druid/indexing/common/task/MoveTask.java | 6 +++ .../indexing/common/task/TaskSerdeTest.java | 53 ++++++++++++++++++- .../indexing/overlord/TaskLifecycleTest.java | 2 - 4 files changed, 63 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java index 97747e211ab..c863742e0ab 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ArchiveTask.java @@ -45,7 +45,11 @@ public class ArchiveTask extends AbstractFixedIntervalTask @JsonProperty("interval") Interval interval ) { - super(id, dataSource, interval); + super( + TaskUtils.makeId(id, "archive", dataSource, interval), + dataSource, + interval + ); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java index 371f0dc38a4..8e628b93188 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MoveTask.java @@ -112,4 +112,10 @@ public class MoveTask extends AbstractFixedIntervalTask return TaskStatus.success(getId()); } + + @JsonProperty + public Map getTargetLoadSpec() + { + return targetLoadSpec; + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 485b3202b54..77ae0af4b52 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -294,7 +294,58 @@ public class TaskSerdeTest Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getInterval(), task2.getInterval()); - Assert.assertEquals(task.getSegments(), ((AppendTask) task2).getSegments()); + Assert.assertEquals(task.getSegments(), task2.getSegments()); + } + + @Test + public void testArchiveTaskSerde() throws Exception + { + final ArchiveTask task = new ArchiveTask( + null, + "foo", + new Interval("2010-01-01/P1D") + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + + Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change + final ArchiveTask task2 = (ArchiveTask) jsonMapper.readValue(json, Task.class); + + Assert.assertEquals("foo", task.getDataSource()); + Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval()); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getInterval(), task2.getInterval()); + } + + @Test + public void testMoveTaskSerde() throws Exception + { + final MoveTask task = new MoveTask( + null, + "foo", + new Interval("2010-01-01/P1D"), + ImmutableMap.of("bucket", "hey", "baseKey", "what") + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + + Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change + final MoveTask task2 = (MoveTask) jsonMapper.readValue(json, Task.class); + + Assert.assertEquals("foo", task.getDataSource()); + Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval()); + Assert.assertEquals(ImmutableMap.of("bucket", "hey", "baseKey", "what"), task.getTargetLoadSpec()); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getInterval(), task2.getInterval()); + Assert.assertEquals(task.getTargetLoadSpec(), task2.getTargetLoadSpec()); } @Test 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 a873daa876f..85637d75c51 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 @@ -78,9 +78,7 @@ import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; import org.easymock.EasyMock; import org.joda.time.DateTime; -import org.joda.time.Duration; import org.joda.time.Interval; -import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; From fc4dd5119497d233e69458177f5e9ecf4a328d70 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Dec 2013 15:33:02 -0800 Subject: [PATCH 36/38] Update versions --- build.sh | 2 +- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Realtime.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 2 +- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.textile | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- services/src/main/java/io/druid/cli/CliCoordinator.java | 2 +- services/src/main/java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/CliHistorical.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliRealtime.java | 2 +- services/src/main/java/io/druid/cli/CliRealtimeExample.java | 2 +- 18 files changed, 23 insertions(+), 23 deletions(-) diff --git a/build.sh b/build.sh index f9bd7723587..9b7478e592e 100755 --- a/build.sh +++ b/build.sh @@ -30,4 +30,4 @@ echo "For examples, see: " echo " " ls -1 examples/*/*sh echo " " -echo "See also http://druid.io/docs/0.6.39" +echo "See also http://druid.io/docs/0.6.40" diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index ded3b7128d9..bb00506700f 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: doc_page --- # Booting a Single Node Cluster # -[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.39-bin.tar.gz). +[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.40-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 00aa3c1d4ff..7aff89ea644 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.39 +git checkout druid-0.6.40 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index 7ce371dfea4..980e1dfb773 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.39"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.40"] druid.zk.service.host=localhost diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 8b5bde905d8..1b35c0979e2 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.39 +cd druid-services-0.6.40 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index ab4d91651ca..50fef2985f6 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -44,7 +44,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h #### Setting up Kafka -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.39/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.40/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 6599031a0b2..348dfd1df53 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.39"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.40"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -238,7 +238,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.39","io.druid.extensions:druid-kafka-seven:0.6.39"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.40","io.druid.extensions:druid-kafka-seven:0.6.40"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index df7b4d2df24..a9605f2fea4 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.39 +cd druid-services-0.6.40 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.textile b/docs/content/Twitter-Tutorial.textile index 6fab02906a1..c82ea981ec4 100644 --- a/docs/content/Twitter-Tutorial.textile +++ b/docs/content/Twitter-Tutorial.textile @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.39-bin.tar.gz. +We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.40-bin.tar.gz. Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index 785f3901eb4..fa2593c63a1 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.39"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.40"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index ae8675c610f..eae65e057f9 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.39","io.druid.extensions:druid-kafka-seven:0.6.39","io.druid.extensions:druid-rabbitmq:0.6.39"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.40","io.druid.extensions:druid-kafka-seven:0.6.40","io.druid.extensions:druid-rabbitmq:0.6.40"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 4b6d6b93721..3a7c88d4426 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -53,7 +53,7 @@ import java.util.List; */ @Command( name = "broker", - description = "Runs a broker node, see http://druid.io/docs/0.6.39/Broker.html for a description" + description = "Runs a broker node, see http://druid.io/docs/0.6.40/Broker.html for a description" ) public class CliBroker extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index ea71bc7de58..486511b1104 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -60,7 +60,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see http://druid.io/docs/0.6.39/Coordinator.html for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.40/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index c9331bd0f04..2cdc9706cef 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -41,7 +41,7 @@ import java.util.List; */ @Command( name = "hadoop", - description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.39/Batch-ingestion.html for a description." + description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.40/Batch-ingestion.html for a description." ) public class CliHadoopIndexer implements Runnable { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 0072e5f3de5..e9c5ece9889 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see http://druid.io/docs/0.6.39/Historical.html for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.40/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index abb516803dc..aa68c14c6bb 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -93,7 +93,7 @@ import java.util.List; */ @Command( name = "overlord", - description = "Runs an Overlord node, see http://druid.io/docs/0.6.39/Indexing-Service.html for a description" + description = "Runs an Overlord node, see http://druid.io/docs/0.6.40/Indexing-Service.html for a description" ) public class CliOverlord extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index e5a68c647c0..70baabbc3c2 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -30,7 +30,7 @@ import java.util.List; */ @Command( name = "realtime", - description = "Runs a realtime node, see http://druid.io/docs/0.6.39/Realtime.html for a description" + description = "Runs a realtime node, see http://druid.io/docs/0.6.40/Realtime.html for a description" ) public class CliRealtime extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 0ecb83e284a..f4b5c4ab5d8 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -42,7 +42,7 @@ import java.util.concurrent.Executor; */ @Command( name = "realtime", - description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.39/Realtime.html for a description" + description = "Runs a standalone realtime node for examples, see http://druid.io/docs/0.6.40/Realtime.html for a description" ) public class CliRealtimeExample extends ServerRunnable { From 231d7a6d1f508b8c9c59fa8dd92a5f7c3e318484 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Dec 2013 15:35:50 -0800 Subject: [PATCH 37/38] [maven-release-plugin] prepare release druid-0.6.40 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 14 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 07074721737..860546c1211 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/common/pom.xml b/common/pom.xml index 0aa0c1221c6..2660c6802d0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/examples/pom.xml b/examples/pom.xml index 0b4f6d94a58..262c481711e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c0d3015bc39..54ce731aaae 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 58b73469c86..11a34c7defc 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f3890bc98ea..b7450874a27 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 2bb456703a6..96588398515 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index b55cc2ad3a1..11193312d9e 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/pom.xml b/pom.xml index 0163a2e34fe..c6d3b367c95 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.40-SNAPSHOT + 0.6.40 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - ${project.artifactId}-${project.version} + druid-0.6.40 diff --git a/processing/pom.xml b/processing/pom.xml index 3baa76d67be..707c405e38b 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index f7264dd1a0a..209eeb1d2d0 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c08f57ef0fc..22576cdf4cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/server/pom.xml b/server/pom.xml index 79fe38908ee..64069421c54 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 diff --git a/services/pom.xml b/services/pom.xml index fd57bbc366d..f20794390b0 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.40-SNAPSHOT + 0.6.40 @@ -90,7 +90,7 @@ - + From d0fd58bbae86c89b490c361e53276bf9f3ec1725 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Dec 2013 15:35:55 -0800 Subject: [PATCH 38/38] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 860546c1211..f28c7dd5221 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 2660c6802d0..eb16f9251ba 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 262c481711e..d21dc9c836f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 54ce731aaae..e5a7af70ad8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 11a34c7defc..7718b246d27 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index b7450874a27..0e25e7c69ed 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 96588398515..08435a100c8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 11193312d9e..a312aa80608 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/pom.xml b/pom.xml index c6d3b367c95..0d792e9bbb7 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.40 + 0.6.41-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.40 + ${project.artifactId}-${project.version} diff --git a/processing/pom.xml b/processing/pom.xml index 707c405e38b..085af80a4da 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 209eeb1d2d0..e9098809f3d 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 22576cdf4cb..7094612ff11 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 64069421c54..576b6060d71 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f20794390b0..7bd5af6e7b4 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.40 + 0.6.41-SNAPSHOT