From 01b7507a5ac6d5759c6eb8632dfb4f0e3e36a5cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Sat, 13 Jul 2013 16:03:46 +0000 Subject: [PATCH 01/68] First version of a RabbitMQ Firehose implementation. --- realtime/pom.xml | 5 + .../realtime/firehose/FirehoseFactory.java | 1 + .../firehose/RabbitMQFirehoseFactory.java | 150 ++++++++++++++++++ 3 files changed, 156 insertions(+) create mode 100644 realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java diff --git a/realtime/pom.xml b/realtime/pom.xml index abe217c1af2..2d59174dd01 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -120,6 +120,11 @@ + + com.rabbitmq + amqp-client + 3.1.1 + diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/FirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/FirehoseFactory.java index 71bf6c108ad..c9d9dfec3de 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/FirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/FirehoseFactory.java @@ -27,6 +27,7 @@ import java.io.IOException; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes({ @JsonSubTypes.Type(name = "kafka-0.7.2", value = KafkaFirehoseFactory.class), + @JsonSubTypes.Type(name = "rabbitmq", value = RabbitMQFirehoseFactory.class), @JsonSubTypes.Type(name = "clipped", value = ClippedFirehoseFactory.class), @JsonSubTypes.Type(name = "timed", value = TimedShutoffFirehoseFactory.class) }) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java new file mode 100644 index 00000000000..e1815f5409a --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java @@ -0,0 +1,150 @@ +package com.metamx.druid.realtime.firehose; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.logger.Logger; +import com.metamx.druid.indexer.data.StringInputRowParser; +import com.metamx.druid.input.InputRow; +import com.rabbitmq.client.*; + +import java.io.IOException; +import java.util.*; + +/** + * A FirehoseFactory + */ +public class RabbitMQFirehoseFactory implements FirehoseFactory{ + + private static final Logger log = new Logger(RabbitMQFirehoseFactory.class); + + @JsonProperty + private final Properties consumerProps; + + @JsonProperty + private final String queue; + + @JsonProperty + private final String exchange; + + @JsonProperty + private final String routingKey; + + @JsonProperty + private final StringInputRowParser parser; + + @JsonCreator + public RabbitMQFirehoseFactory( + @JsonProperty("consumerProps") Properties consumerProps, + @JsonProperty("queue") String queue, + @JsonProperty("exchange") String exchange, + @JsonProperty("routingKey") String routingKey, + @JsonProperty("parser") StringInputRowParser parser + ) + { + this.consumerProps = consumerProps; + this.queue = queue; + this.exchange = exchange; + this.routingKey = routingKey; + this.parser = parser; + + parser.addDimensionExclusion("queue"); + parser.addDimensionExclusion("exchange"); + parser.addDimensionExclusion("routingKey"); + } + + @Override + public Firehose connect() throws IOException { + + final ConnectionFactory factory = new ConnectionFactory(); + factory.setHost(consumerProps.getProperty("host", factory.getHost())); + factory.setUsername(consumerProps.getProperty("username", factory.getUsername())); + factory.setPassword(consumerProps.getProperty("password", factory.getPassword())); + factory.setVirtualHost(consumerProps.getProperty("virtualHost", factory.getVirtualHost())); + + final Connection connection = factory.newConnection(); + final Channel channel = connection.createChannel(); + channel.queueDeclare(queue, true, false, false, null); + channel.queueBind(queue, exchange, routingKey); + final QueueingConsumer consumer = new QueueingConsumer(channel); + channel.basicConsume(queue, false, consumer); + + return new Firehose(){ + + //private final Connection connection = conn; + //private final Channel channel = ch; + //private final QueueingConsumer consumer = qc; + + /** + * Storing the latest delivery as a member variable should be safe since this will only be run + * by a single thread. + */ + private QueueingConsumer.Delivery delivery; + + /** + * Store the latest delivery tag to be able to commit (acknowledge) the message delivery up to + * and including this tag. See commit() for more detail. + */ + private long lastDeliveryTag; + + @Override + public boolean hasMore() { + try { + delivery = consumer.nextDelivery(); + lastDeliveryTag = delivery.getEnvelope().getDeliveryTag(); + + log.debug("got new message"); + } catch (InterruptedException e) { + log.wtf(e, "Don't know if this is supposed to ever happen."); + return false; + } + + if(delivery != null){ + return true; + } + + // Shouldn't ever get here but in case we'll assume there is no more stuff. + return false; + } + + @Override + public InputRow nextRow() { + log.debug("consuming new message"); + + return parser.parse(new String(delivery.getBody())); + } + + @Override + public Runnable commit() { + + // This method will be called from the same thread that calls the other methods of + // this Firehose. However, the returned Runnable will be called by a different thread. + // + // It should be (thread) safe to copy the lastDeliveryTag like we do below. + return new Runnable() + { + // Store (copy) the last delivery tag to "become" thread safe. + final long deliveryTag = lastDeliveryTag; + + @Override + public void run() + { + try { + log.info("Acknowledging delivery of messages up to tag: " + deliveryTag); + + // Acknowledge all messages up to and including the stored delivery tag. + channel.basicAck(deliveryTag, true); + } catch (IOException e) { + log.error(e, "Unable to acknowledge message reception to message queue."); + } + } + }; + } + + @Override + public void close() throws IOException { + channel.close(); + connection.close(); + } + }; + } +} From fc74333a28ddab0495c0875ae9c3511081633772 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Sat, 13 Jul 2013 16:14:10 +0000 Subject: [PATCH 02/68] adding an example dir to test the rabbitmq implementation --- examples/bin/examples/rabbitmq/query.body | 19 ++++++++++ .../examples/rabbitmq/rabbitmq_realtime.spec | 36 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 examples/bin/examples/rabbitmq/query.body create mode 100644 examples/bin/examples/rabbitmq/rabbitmq_realtime.spec diff --git a/examples/bin/examples/rabbitmq/query.body b/examples/bin/examples/rabbitmq/query.body new file mode 100644 index 00000000000..05007c955b9 --- /dev/null +++ b/examples/bin/examples/rabbitmq/query.body @@ -0,0 +1,19 @@ +{ + "queryType": "groupBy", + "dataSource": "randSeq", + "granularity": "all", + "dimensions": [], + "aggregations":[ + { "type": "count", "name": "rows"}, + { "type": "doubleSum", "fieldName": "events", "name": "e"}, + { "type": "doubleSum", "fieldName": "outColumn", "name": "randomNumberSum"} + ], + "postAggregations":[ + { "type":"arithmetic", + "name":"avg_random", + "fn":"/", + "fields":[ {"type":"fieldAccess","name":"randomNumberSum","fieldName":"randomNumberSum"}, + {"type":"fieldAccess","name":"rows","fieldName":"rows"} ]} + ], + "intervals":["2012-10-01T00:00/2020-01-01T00"] +} diff --git a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec new file mode 100644 index 00000000000..25ce1182554 --- /dev/null +++ b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec @@ -0,0 +1,36 @@ +[{ + "schema": { + "dataSource": "randseq", + "aggregators": [ + {"type": "count", "name": "events"}, + {"type": "doubleSum", "name": "outColumn", "fieldName": "inColumn"} + ], + "indexGranularity": "minute", + "shardSpec": {"type": "none"} + }, + + "config": { + "maxRowsInMemory": 50000, + "intermediatePersistPeriod": "PT1m" + }, + + "firehose" : { "type" : "rabbitmq", + "consumerProps" : { "username": "test-dude", + "password": "test-word", + "virtualHost": "test-vhost", + "host": "localhost" + }, + "queue" : "druidtest", + "exchange": "test-exchange", + "routingKey": "#", + "parser" : { "timestampSpec" : { "column" : "utcdt", "format" : "iso" }, + "data" : { "format" : "json" }, + "dimensionExclusions" : ["wp"] } }, + + "plumber": { + "type": "realtime", + "windowPeriod": "PT5m", + "segmentGranularity": "hour", + "basePersistDirectory": "/tmp/example/rand_realtime/basePersist" + } +}] From 4d998e4c27b34c43dacbb4f4f2816c31567dfe5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Sat, 13 Jul 2013 16:49:39 +0000 Subject: [PATCH 03/68] Making more things configurable through the realtime.spec file. --- examples/bin/examples/rabbitmq/rabbitmq_realtime.spec | 6 +++++- .../realtime/firehose/RabbitMQFirehoseFactory.java | 10 ++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec index 25ce1182554..0b1f1950af4 100644 --- a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec +++ b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec @@ -18,7 +18,11 @@ "consumerProps" : { "username": "test-dude", "password": "test-word", "virtualHost": "test-vhost", - "host": "localhost" + "host": "localhost", + "durable": "true", + "exclusive": "false", + "autoDelete": "false", + "audoAck": "false" }, "queue" : "druidtest", "exchange": "test-exchange", diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java index e1815f5409a..6140f82a93f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java @@ -61,12 +61,17 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ factory.setPassword(consumerProps.getProperty("password", factory.getPassword())); factory.setVirtualHost(consumerProps.getProperty("virtualHost", factory.getVirtualHost())); + boolean durable = Boolean.valueOf(consumerProps.getProperty("durable", "false")); + boolean exclusive = Boolean.valueOf(consumerProps.getProperty("exclusive", "false")); + boolean autoDelete = Boolean.valueOf(consumerProps.getProperty("autoDelete", "false")); + boolean autoAck = Boolean.valueOf(consumerProps.getProperty("autoAck", "true")); + final Connection connection = factory.newConnection(); final Channel channel = connection.createChannel(); - channel.queueDeclare(queue, true, false, false, null); + channel.queueDeclare(queue, durable, exclusive, autoDelete, null); channel.queueBind(queue, exchange, routingKey); final QueueingConsumer consumer = new QueueingConsumer(channel); - channel.basicConsume(queue, false, consumer); + channel.basicConsume(queue, autoAck, consumer); return new Firehose(){ @@ -103,6 +108,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ } // Shouldn't ever get here but in case we'll assume there is no more stuff. + log.wtf("We shouldn't be here!"); return false; } From 13328a6b36410edf89e3e8a59a36fb20e43e018b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Mon, 15 Jul 2013 09:50:58 +0000 Subject: [PATCH 04/68] Implemented a better query.body for the test I'm using. Added a shutdown listener to the connection and channel to get a notification when they are closed. Question remains whether the connection should be re-established when that happens. --- examples/bin/examples/rabbitmq/query.body | 19 ++++++------------- .../firehose/RabbitMQFirehoseFactory.java | 15 +++++++++++++++ 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/examples/bin/examples/rabbitmq/query.body b/examples/bin/examples/rabbitmq/query.body index 05007c955b9..59ca8fe1e2c 100644 --- a/examples/bin/examples/rabbitmq/query.body +++ b/examples/bin/examples/rabbitmq/query.body @@ -1,19 +1,12 @@ { "queryType": "groupBy", - "dataSource": "randSeq", + "dataSource": "rabbitmqtest", "granularity": "all", "dimensions": [], - "aggregations":[ - { "type": "count", "name": "rows"}, - { "type": "doubleSum", "fieldName": "events", "name": "e"}, - { "type": "doubleSum", "fieldName": "outColumn", "name": "randomNumberSum"} + "aggregations": [ + { "type": "count", "name": "rows" }, + {"type": "longSum", "name": "imps", "fieldName": "impressions"}, + {"type": "doubleSum", "name": "wp", "fieldName": "wp"} ], - "postAggregations":[ - { "type":"arithmetic", - "name":"avg_random", - "fn":"/", - "fields":[ {"type":"fieldAccess","name":"randomNumberSum","fieldName":"randomNumberSum"}, - {"type":"fieldAccess","name":"rows","fieldName":"rows"} ]} - ], - "intervals":["2012-10-01T00:00/2020-01-01T00"] + "intervals": ["2010-01-01T00:00/2020-01-01T00"] } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java index 6140f82a93f..b03c7c73f32 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java @@ -72,6 +72,20 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ channel.queueBind(queue, exchange, routingKey); final QueueingConsumer consumer = new QueueingConsumer(channel); channel.basicConsume(queue, autoAck, consumer); + channel.addShutdownListener(new ShutdownListener() { + @Override + public void shutdownCompleted(ShutdownSignalException cause) { + log.warn(cause, "Channel closed!"); + //TODO: should we re-establish the connection here? + } + }); + connection.addShutdownListener(new ShutdownListener() { + @Override + public void shutdownCompleted(ShutdownSignalException cause) { + log.warn(cause, "Connection closed!"); + //TODO: should we re-establish the connection here? + } + }); return new Firehose(){ @@ -148,6 +162,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ @Override public void close() throws IOException { + log.info("Closing connection to RabbitMQ"); channel.close(); connection.close(); } From 82d2623cc97776028c9b5c01691c6de4872bd222 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Mon, 15 Jul 2013 15:25:59 +0000 Subject: [PATCH 05/68] Minor sanity checks when checking for and processing a new message. --- .../firehose/RabbitMQFirehoseFactory.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java index b03c7c73f32..7200db46bad 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java @@ -89,10 +89,6 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ return new Firehose(){ - //private final Connection connection = conn; - //private final Channel channel = ch; - //private final QueueingConsumer consumer = qc; - /** * Storing the latest delivery as a member variable should be safe since this will only be run * by a single thread. @@ -107,28 +103,35 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ @Override public boolean hasMore() { + delivery = null; try { delivery = consumer.nextDelivery(); lastDeliveryTag = delivery.getEnvelope().getDeliveryTag(); log.debug("got new message"); } catch (InterruptedException e) { + //TODO: I'm not exactly sure how we should react to this. + // Does it mean that delivery will be null and we should handle that + // as if there are no more messages (return false)? log.wtf(e, "Don't know if this is supposed to ever happen."); - return false; } if(delivery != null){ + // If delivery is non-null, we report that there is something more to process. return true; } - // Shouldn't ever get here but in case we'll assume there is no more stuff. - log.wtf("We shouldn't be here!"); + // This means that delivery is null so we have nothing more to process. return false; } @Override public InputRow nextRow() { - log.debug("consuming new message"); + if(delivery == null){ + //Just making sure. + log.wtf("I have nothing in delivery. Method hasMore() should have returned false."); + return null; + } return parser.parse(new String(delivery.getBody())); } From a43ccf587ff890b1cad71483d0de7fb7a35d31ed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Mon, 15 Jul 2013 18:38:21 +0000 Subject: [PATCH 06/68] Fixed some errors in the realtime.spec and query.body files and cleaned up the format to make it more readable as an example. --- examples/bin/examples/rabbitmq/query.body | 2 +- .../examples/rabbitmq/rabbitmq_realtime.spec | 73 ++++++++++--------- 2 files changed, 39 insertions(+), 36 deletions(-) diff --git a/examples/bin/examples/rabbitmq/query.body b/examples/bin/examples/rabbitmq/query.body index 59ca8fe1e2c..43623cfa733 100644 --- a/examples/bin/examples/rabbitmq/query.body +++ b/examples/bin/examples/rabbitmq/query.body @@ -8,5 +8,5 @@ {"type": "longSum", "name": "imps", "fieldName": "impressions"}, {"type": "doubleSum", "name": "wp", "fieldName": "wp"} ], - "intervals": ["2010-01-01T00:00/2020-01-01T00"] + "intervals": ["2010-01-01T00:00/2020-01-01T00:00"] } diff --git a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec index 0b1f1950af4..63dfb7496d3 100644 --- a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec +++ b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec @@ -1,40 +1,43 @@ [{ - "schema": { - "dataSource": "randseq", - "aggregators": [ - {"type": "count", "name": "events"}, - {"type": "doubleSum", "name": "outColumn", "fieldName": "inColumn"} + "schema" : { + "dataSource":"rabbitmqtest", + "aggregators":[ + {"type":"count", "name":"impressions"}, + {"type":"doubleSum","name":"wp","fieldName":"wp"} ], - "indexGranularity": "minute", - "shardSpec": {"type": "none"} + "indexGranularity":"minute", + "shardSpec" : { "type": "none" } }, - - "config": { - "maxRowsInMemory": 50000, - "intermediatePersistPeriod": "PT1m" + "config" : { + "maxRowsInMemory" : 500000, + "intermediatePersistPeriod" : "PT1m" }, - - "firehose" : { "type" : "rabbitmq", - "consumerProps" : { "username": "test-dude", - "password": "test-word", - "virtualHost": "test-vhost", - "host": "localhost", - "durable": "true", - "exclusive": "false", - "autoDelete": "false", - "audoAck": "false" - }, - "queue" : "druidtest", - "exchange": "test-exchange", - "routingKey": "#", - "parser" : { "timestampSpec" : { "column" : "utcdt", "format" : "iso" }, - "data" : { "format" : "json" }, - "dimensionExclusions" : ["wp"] } }, - - "plumber": { - "type": "realtime", - "windowPeriod": "PT5m", - "segmentGranularity": "hour", - "basePersistDirectory": "/tmp/example/rand_realtime/basePersist" + "firehose" : { + "type" : "rabbitmq", + "consumerProps" : { + "host": "localhost", + "username": "test-dude", + "password": "test-word", + "virtualHost": "test-vhost", + "durable": "true", + "exclusive": "false", + "autoDelete": "false", + "autoAck": "false" + }, + "exchange": "test-exchange", + "queue" : "druidtest", + "routingKey": "#", + "parser" : { + "timestampSpec" : { "column" : "utcdt", "format" : "iso" }, + "data" : { "format" : "json" }, + "dimensionExclusions" : ["wp"] + } + }, + "plumber" : { + "type" : "realtime", + "windowPeriod" : "PT5m", + "segmentGranularity":"hour", + "basePersistDirectory" : "/tmp/realtime/basePersist", + "rejectionPolicy": { "type": "messageTime" } } -}] +}] \ No newline at end of file From dcc9942d8a9a814993a2727ce21217b53f3c81d0 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 15 Jul 2013 14:19:05 -0700 Subject: [PATCH 07/68] 1) Update version of http-client --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index dd042d33c33..40b4a66218e 100644 --- a/pom.xml +++ b/pom.xml @@ -65,7 +65,7 @@ com.metamx http-client - 0.7.1 + 0.8.1 com.metamx From 945bc9a3707d4d9036c83d4d3d4932f7a361c6f9 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 16 Jul 2013 08:02:49 -0700 Subject: [PATCH 08/68] Limited retries on s3 segment puller exceptions Useful for reducing spurious alerts due to general s3 flakiness. --- .../druid/loading/S3DataSegmentPuller.java | 148 +++++++++++++----- 1 file changed, 109 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java index 011e1633ca1..210decc1178 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -19,6 +19,7 @@ package com.metamx.druid.loading; +import com.google.common.base.Throwables; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.common.io.Files; @@ -30,16 +31,17 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.common.s3.S3Utils; import com.metamx.druid.utils.CompressionUtils; import org.apache.commons.io.FileUtils; -import org.jets3t.service.S3ServiceException; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; +import org.jets3t.service.model.StorageObject; import java.io.File; import java.io.IOException; import java.io.InputStream; import java.util.Map; +import java.util.Random; +import java.util.concurrent.Callable; import java.util.zip.GZIPInputStream; /** @@ -62,9 +64,9 @@ public class S3DataSegmentPuller implements DataSegmentPuller } @Override - public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException + public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException { - S3Coords s3Coords = new S3Coords(segment); + final S3Coords s3Coords = new S3Coords(segment); log.info("Pulling index at path[%s] to outDir[%s]", s3Coords, outDir); @@ -80,41 +82,52 @@ public class S3DataSegmentPuller implements DataSegmentPuller throw new ISE("outDir[%s] must be a directory.", outDir); } - long startTime = System.currentTimeMillis(); - S3Object s3Obj = null; - try { - s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + retryS3Operation( + new Callable() + { + @Override + public Void call() throws Exception + { + long startTime = System.currentTimeMillis(); + S3Object s3Obj = null; - InputStream in = null; - try { - in = s3Obj.getDataInputStream(); - final String key = s3Obj.getKey(); - if (key.endsWith(".zip")) { - CompressionUtils.unzip(in, outDir); - } else if (key.endsWith(".gz")) { - final File outFile = new File(outDir, toFilename(key, ".gz")); - ByteStreams.copy(new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); - } else { - ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); - } - log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); - } - catch (IOException e) { - FileUtils.deleteDirectory(outDir); - throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); - } - finally { - Closeables.closeQuietly(in); - } + try { + s3Obj = s3Client.getObject(s3Coords.bucket, s3Coords.path); + + InputStream in = null; + try { + in = s3Obj.getDataInputStream(); + final String key = s3Obj.getKey(); + if (key.endsWith(".zip")) { + CompressionUtils.unzip(in, outDir); + } else if (key.endsWith(".gz")) { + final File outFile = new File(outDir, toFilename(key, ".gz")); + ByteStreams.copy(new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); + } else { + ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); + } + log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); + return null; + } + catch (IOException e) { + FileUtils.deleteDirectory(outDir); + throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); + } + finally { + Closeables.closeQuietly(in); + } + } + finally { + S3Utils.closeStreamsQuietly(s3Obj); + } + } + } + ); } catch (Exception e) { throw new SegmentLoadingException(e, e.getMessage()); } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - } - } private String toFilename(String key, final String suffix) @@ -124,29 +137,86 @@ public class S3DataSegmentPuller implements DataSegmentPuller return filename; } - private boolean isObjectInBucket(S3Coords coords) throws SegmentLoadingException + private boolean isObjectInBucket(final S3Coords coords) throws SegmentLoadingException { try { - return s3Client.isObjectInBucket(coords.bucket, coords.path); + return retryS3Operation( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } + } + ); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); } catch (ServiceException e) { - throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); + throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); } } @Override public long getLastModified(DataSegment segment) throws SegmentLoadingException { - S3Coords coords = new S3Coords(segment); + final S3Coords coords = new S3Coords(segment); try { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + final StorageObject objDetails = retryS3Operation( + new Callable() + { + @Override + public StorageObject call() throws Exception + { + return s3Client.getObjectDetails(coords.bucket, coords.path); + } + } + ); return objDetails.getLastModifiedDate().getTime(); } - catch (S3ServiceException e) { + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + catch (ServiceException e) { throw new SegmentLoadingException(e, e.getMessage()); } } + private T retryS3Operation(Callable f) throws ServiceException, InterruptedException + { + int nTry = 0; + final int maxTries = 3; + final long baseSleepMillis = 1000; + final double fuzziness = 0.2; + while (true) { + try { + nTry++; + return f.call(); + } + catch (ServiceException e) { + if (nTry <= maxTries && + (e.getCause() instanceof IOException || + (e.getErrorCode() != null && e.getErrorCode().equals("RequestTimeout")))) { + // Retryable + final long sleepMillis = Math.max( + baseSleepMillis, + (long) (baseSleepMillis * Math.pow(2, nTry) * + (1 + new Random().nextGaussian() * fuzziness)) + ); + log.info(e, "S3 fail on try %d/%d, retrying in %,dms.", nTry, maxTries, sleepMillis); + Thread.sleep(sleepMillis); + } else { + throw e; + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + private static class S3Coords { String bucket; From 320f1fe84051d0abb6cd200a46d820a7810a59db Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 16 Jul 2013 08:07:14 -0700 Subject: [PATCH 09/68] Move retryS3Operation to S3Utils --- .../com/metamx/druid/common/s3/S3Utils.java | 41 +++++++++++++++++++ .../druid/loading/S3DataSegmentPuller.java | 39 ++---------------- 2 files changed, 44 insertions(+), 36 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java index 079f46676ca..dee3bf320f5 100644 --- a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java +++ b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java @@ -19,8 +19,10 @@ package com.metamx.druid.common.s3; +import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; @@ -28,6 +30,8 @@ import org.jets3t.service.model.S3Object; import java.io.File; import java.io.IOException; import java.security.NoSuchAlgorithmException; +import java.util.Random; +import java.util.concurrent.Callable; /** * @@ -80,4 +84,41 @@ public class S3Utils } } + + /** + * Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not + * found, etc) are not retried. + */ + public static T retryS3Operation(Callable f) throws ServiceException, InterruptedException + { + int nTry = 0; + final int maxTries = 3; + final long baseSleepMillis = 1000; + final double fuzziness = 0.2; + while (true) { + try { + nTry++; + return f.call(); + } + catch (ServiceException e) { + if (nTry <= maxTries && + (e.getCause() instanceof IOException || + (e.getErrorCode() != null && e.getErrorCode().equals("RequestTimeout")))) { + // Retryable + final long sleepMillis = Math.max( + baseSleepMillis, + (long) (baseSleepMillis * Math.pow(2, nTry) * + (1 + new Random().nextGaussian() * fuzziness)) + ); + log.info(e, "S3 fail on try %d/%d, retrying in %,dms.", nTry, maxTries, sleepMillis); + Thread.sleep(sleepMillis); + } else { + throw e; + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java index 210decc1178..59100b5fdeb 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -83,7 +83,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller } try { - retryS3Operation( + S3Utils.retryS3Operation( new Callable() { @Override @@ -140,7 +140,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller private boolean isObjectInBucket(final S3Coords coords) throws SegmentLoadingException { try { - return retryS3Operation( + return S3Utils.retryS3Operation( new Callable() { @Override @@ -164,7 +164,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller { final S3Coords coords = new S3Coords(segment); try { - final StorageObject objDetails = retryS3Operation( + final StorageObject objDetails = S3Utils.retryS3Operation( new Callable() { @Override @@ -184,39 +184,6 @@ public class S3DataSegmentPuller implements DataSegmentPuller } } - private T retryS3Operation(Callable f) throws ServiceException, InterruptedException - { - int nTry = 0; - final int maxTries = 3; - final long baseSleepMillis = 1000; - final double fuzziness = 0.2; - while (true) { - try { - nTry++; - return f.call(); - } - catch (ServiceException e) { - if (nTry <= maxTries && - (e.getCause() instanceof IOException || - (e.getErrorCode() != null && e.getErrorCode().equals("RequestTimeout")))) { - // Retryable - final long sleepMillis = Math.max( - baseSleepMillis, - (long) (baseSleepMillis * Math.pow(2, nTry) * - (1 + new Random().nextGaussian() * fuzziness)) - ); - log.info(e, "S3 fail on try %d/%d, retrying in %,dms.", nTry, maxTries, sleepMillis); - Thread.sleep(sleepMillis); - } else { - throw e; - } - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - private static class S3Coords { String bucket; From 4201d9ff246ce35a89832f7ee0a66bb91857175f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 16 Jul 2013 08:08:25 -0700 Subject: [PATCH 10/68] Remove unused methods from S3Utils --- .../com/metamx/druid/common/s3/S3Utils.java | 31 ------------------- 1 file changed, 31 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java index dee3bf320f5..f0ba2d133d3 100644 --- a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java +++ b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java @@ -40,37 +40,6 @@ public class S3Utils { private static final Logger log = new Logger(S3Utils.class); - public static void putFileToS3( - File localFile, RestS3Service s3Client, String outputS3Bucket, String outputS3Path - ) - throws S3ServiceException, IOException, NoSuchAlgorithmException - { - S3Object s3Obj = new S3Object(localFile); - s3Obj.setBucketName(outputS3Bucket); - s3Obj.setKey(outputS3Path); - - log.info("Uploading file[%s] to [s3://%s/%s]", localFile, s3Obj.getBucketName(), s3Obj.getKey()); - s3Client.putObject(new S3Bucket(outputS3Bucket), s3Obj); - } - - public static void putFileToS3WrapExceptions( - File localFile, RestS3Service s3Client, String outputS3Bucket, String outputS3Path - ) - { - try { - putFileToS3(localFile, s3Client, outputS3Bucket, outputS3Path); - } - catch (S3ServiceException e) { - throw new RuntimeException(e); - } - catch (IOException e) { - throw new RuntimeException(e); - } - catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e); - } - } - public static void closeStreamsQuietly(S3Object s3Obj) { if (s3Obj == null) { From 22ec7c53fc8b65f9839dbff56bf983937fbe092f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Tue, 16 Jul 2013 15:08:39 +0000 Subject: [PATCH 11/68] Code cleanup and formatting, documentation and moving all configuration into the consumerProps. --- .../examples/rabbitmq/rabbitmq_realtime.spec | 9 +- .../firehose/RabbitMQFirehoseFactory.java | 165 ++++++++++++------ 2 files changed, 112 insertions(+), 62 deletions(-) diff --git a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec index 63dfb7496d3..50f892162aa 100644 --- a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec +++ b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec @@ -19,14 +19,13 @@ "username": "test-dude", "password": "test-word", "virtualHost": "test-vhost", + "exchange": "test-exchange", + "queue" : "druidtest", + "routingKey": "#", "durable": "true", "exclusive": "false", - "autoDelete": "false", - "autoAck": "false" + "autoDelete": "false" }, - "exchange": "test-exchange", - "queue" : "druidtest", - "routingKey": "#", "parser" : { "timestampSpec" : { "column" : "utcdt", "format" : "iso" }, "data" : { "format" : "json" }, diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java index 7200db46bad..36dbad4d106 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java @@ -8,87 +8,133 @@ import com.metamx.druid.input.InputRow; import com.rabbitmq.client.*; import java.io.IOException; -import java.util.*; +import java.util.Properties; /** - * A FirehoseFactory + * A FirehoseFactory for RabbitMQ. + *

+ * It will receive it's configuration through the realtime.spec file and expects to find a + * consumerProps element in the firehose definition with values for a number of configuration options. + * Below is a complete example for a RabbitMQ firehose configuration with some explanation. Options + * that have defaults can be skipped but options with no defaults must be specified with the exception + * of the URI property. If the URI property is set, it will override any other property that was also + * set. + *

+ * File: realtime.spec + *

+ *   "firehose" : {
+ *     "type" : "rabbitmq",
+ *     "consumerProps" : {
+ *       "host": "localhost",                 # The hostname of the RabbitMQ broker to connect to. Default: 'localhost'
+ *       "port": "5672",                      # The port number to connect to on the RabbitMQ broker. Default: '5672'
+ *       "username": "test-dude",             # The username to use to connect to RabbitMQ. Default: 'guest'
+ *       "password": "test-word",             # The password to use to connect to RabbitMQ. Default: 'guest'
+ *       "virtualHost": "test-vhost",         # The virtual host to connect to. Default: '/'
+ *       "uri": "amqp://mqserver:1234/vhost", # The URI string to use to connect to RabbitMQ. No default and not needed
+ *       "exchange": "test-exchange",         # The exchange to connect to. No default
+ *       "queue" : "druidtest",               # The queue to connect to or create. No default
+ *       "routingKey": "#",                   # The routing key to use to bind the queue to the exchange. No default
+ *       "durable": "true",                   # Whether the queue should be durable. Default: 'false'
+ *       "exclusive": "false",                # Whether the queue should be exclusive. Default: 'false'
+ *       "autoDelete": "false"                # Whether the queue should auto-delete on disconnect. Default: 'false'
+ *     },
+ *     "parser" : {
+ *       "timestampSpec" : { "column" : "utcdt", "format" : "iso" },
+ *       "data" : { "format" : "json" },
+ *       "dimensionExclusions" : ["wp"]
+ *     }
+ *   },
+ * 
+ *

+ * Limitations: This implementation will not attempt to reconnect to the MQ broker if the + * connection to it is lost. Furthermore it does not support any automatic failover on high availability + * RabbitMQ clusters. This is not supported by the underlying AMQP client library and while the behavior + * could be "faked" to some extent we haven't implemented that yet. However, if a policy is defined in + * the RabbitMQ cluster that sets the "ha-mode" and "ha-sync-mode" properly on the queue that this + * Firehose connects to, messages should survive an MQ broker node failure and be delivered once a + * connection to another node is set up. + *

+ * For more information on RabbitMQ high availability please see: + * http://www.rabbitmq.com/ha.html. */ -public class RabbitMQFirehoseFactory implements FirehoseFactory{ - +public class RabbitMQFirehoseFactory implements FirehoseFactory +{ private static final Logger log = new Logger(RabbitMQFirehoseFactory.class); - @JsonProperty private final Properties consumerProps; - - @JsonProperty - private final String queue; - - @JsonProperty - private final String exchange; - - @JsonProperty - private final String routingKey; - @JsonProperty private final StringInputRowParser parser; @JsonCreator public RabbitMQFirehoseFactory( @JsonProperty("consumerProps") Properties consumerProps, - @JsonProperty("queue") String queue, - @JsonProperty("exchange") String exchange, - @JsonProperty("routingKey") String routingKey, @JsonProperty("parser") StringInputRowParser parser ) { this.consumerProps = consumerProps; - this.queue = queue; - this.exchange = exchange; - this.routingKey = routingKey; this.parser = parser; - - parser.addDimensionExclusion("queue"); - parser.addDimensionExclusion("exchange"); - parser.addDimensionExclusion("routingKey"); } @Override - public Firehose connect() throws IOException { - + public Firehose connect() throws IOException + { final ConnectionFactory factory = new ConnectionFactory(); factory.setHost(consumerProps.getProperty("host", factory.getHost())); + factory.setPort(Integer.parseInt(consumerProps.getProperty("port", Integer.toString(factory.getPort())))); factory.setUsername(consumerProps.getProperty("username", factory.getUsername())); factory.setPassword(consumerProps.getProperty("password", factory.getPassword())); factory.setVirtualHost(consumerProps.getProperty("virtualHost", factory.getVirtualHost())); + // If the URI property has a value it overrides the values set above. + if(consumerProps.containsKey("uri")){ + try { + factory.setUri(consumerProps.getProperty("uri")); + } + catch(Exception e){ + // A little silly to throw an IOException but we'll make do for now with it. + throw new IOException("Bad URI format.", e); + } + } + + String queue = consumerProps.getProperty("queue"); + String exchange = consumerProps.getProperty("exchange"); + String routingKey = consumerProps.getProperty("routingKey"); + boolean durable = Boolean.valueOf(consumerProps.getProperty("durable", "false")); boolean exclusive = Boolean.valueOf(consumerProps.getProperty("exclusive", "false")); boolean autoDelete = Boolean.valueOf(consumerProps.getProperty("autoDelete", "false")); - boolean autoAck = Boolean.valueOf(consumerProps.getProperty("autoAck", "true")); final Connection connection = factory.newConnection(); - final Channel channel = connection.createChannel(); - channel.queueDeclare(queue, durable, exclusive, autoDelete, null); - channel.queueBind(queue, exchange, routingKey); - final QueueingConsumer consumer = new QueueingConsumer(channel); - channel.basicConsume(queue, autoAck, consumer); - channel.addShutdownListener(new ShutdownListener() { + connection.addShutdownListener(new ShutdownListener() + { @Override - public void shutdownCompleted(ShutdownSignalException cause) { - log.warn(cause, "Channel closed!"); - //TODO: should we re-establish the connection here? - } - }); - connection.addShutdownListener(new ShutdownListener() { - @Override - public void shutdownCompleted(ShutdownSignalException cause) { + public void shutdownCompleted(ShutdownSignalException cause) + { log.warn(cause, "Connection closed!"); //TODO: should we re-establish the connection here? } }); - return new Firehose(){ + final Channel channel = connection.createChannel(); + channel.queueDeclare(queue, durable, exclusive, autoDelete, null); + channel.queueBind(queue, exchange, routingKey); + channel.addShutdownListener(new ShutdownListener() + { + @Override + public void shutdownCompleted(ShutdownSignalException cause) + { + log.warn(cause, "Channel closed!"); + //TODO: should we re-establish the connection here? + } + }); + // We create a QueueingConsumer that will not auto-acknowledge messages since that + // happens on commit(). + final QueueingConsumer consumer = new QueueingConsumer(channel); + channel.basicConsume(queue, false, consumer); + + return new Firehose() + { /** * Storing the latest delivery as a member variable should be safe since this will only be run * by a single thread. @@ -102,21 +148,22 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ private long lastDeliveryTag; @Override - public boolean hasMore() { + public boolean hasMore() + { delivery = null; try { delivery = consumer.nextDelivery(); lastDeliveryTag = delivery.getEnvelope().getDeliveryTag(); - - log.debug("got new message"); - } catch (InterruptedException e) { - //TODO: I'm not exactly sure how we should react to this. + //log.debug("Received new message from RabbitMQ"); + } + catch (InterruptedException e) { + //TODO: Not exactly sure how we should react to this. // Does it mean that delivery will be null and we should handle that // as if there are no more messages (return false)? - log.wtf(e, "Don't know if this is supposed to ever happen."); + log.wtf(e, "Got interrupted while waiting for next delivery. Doubt this should ever happen."); } - if(delivery != null){ + if (delivery != null) { // If delivery is non-null, we report that there is something more to process. return true; } @@ -126,8 +173,9 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ } @Override - public InputRow nextRow() { - if(delivery == null){ + public InputRow nextRow() + { + if (delivery == null) { //Just making sure. log.wtf("I have nothing in delivery. Method hasMore() should have returned false."); return null; @@ -137,12 +185,13 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ } @Override - public Runnable commit() { - + public Runnable commit() + { // This method will be called from the same thread that calls the other methods of // this Firehose. However, the returned Runnable will be called by a different thread. // - // It should be (thread) safe to copy the lastDeliveryTag like we do below. + // It should be (thread) safe to copy the lastDeliveryTag like we do below and then + // acknowledge values up to and including that value. return new Runnable() { // Store (copy) the last delivery tag to "become" thread safe. @@ -156,7 +205,8 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ // Acknowledge all messages up to and including the stored delivery tag. channel.basicAck(deliveryTag, true); - } catch (IOException e) { + } + catch (IOException e) { log.error(e, "Unable to acknowledge message reception to message queue."); } } @@ -164,7 +214,8 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory{ } @Override - public void close() throws IOException { + public void close() throws IOException + { log.info("Closing connection to RabbitMQ"); channel.close(); connection.close(); From 217d539e46f35ad019004e5117a98285e589d9fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Tue, 16 Jul 2013 15:18:03 +0000 Subject: [PATCH 12/68] Adding a newline at the end of the file. --- examples/bin/examples/rabbitmq/rabbitmq_realtime.spec | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec index 50f892162aa..589518d8edd 100644 --- a/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec +++ b/examples/bin/examples/rabbitmq/rabbitmq_realtime.spec @@ -39,4 +39,4 @@ "basePersistDirectory" : "/tmp/realtime/basePersist", "rejectionPolicy": { "type": "messageTime" } } -}] \ No newline at end of file +}] From 6258d77398f1b603cf47d59d6d314f4724d6555f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 16 Jul 2013 08:25:19 -0700 Subject: [PATCH 13/68] Retry s3 operations on non-wrapped IOExceptions Can happen if we get a socket related mishap while fetching an s3 object. --- .../com/metamx/druid/common/s3/S3Utils.java | 36 +++++++++++-------- .../druid/loading/S3DataSegmentPuller.java | 2 +- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java index f0ba2d133d3..15fa7c88240 100644 --- a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java +++ b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java @@ -21,15 +21,10 @@ package com.metamx.druid.common.s3; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; -import org.jets3t.service.S3ServiceException; import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; -import java.io.File; import java.io.IOException; -import java.security.NoSuchAlgorithmException; import java.util.Random; import java.util.concurrent.Callable; @@ -62,25 +57,23 @@ public class S3Utils { int nTry = 0; final int maxTries = 3; - final long baseSleepMillis = 1000; - final double fuzziness = 0.2; while (true) { try { nTry++; return f.call(); } + catch (IOException e) { + if (nTry <= maxTries) { + awaitNextRetry(e, nTry); + } else { + throw Throwables.propagate(e); + } + } catch (ServiceException e) { if (nTry <= maxTries && (e.getCause() instanceof IOException || (e.getErrorCode() != null && e.getErrorCode().equals("RequestTimeout")))) { - // Retryable - final long sleepMillis = Math.max( - baseSleepMillis, - (long) (baseSleepMillis * Math.pow(2, nTry) * - (1 + new Random().nextGaussian() * fuzziness)) - ); - log.info(e, "S3 fail on try %d/%d, retrying in %,dms.", nTry, maxTries, sleepMillis); - Thread.sleep(sleepMillis); + awaitNextRetry(e, nTry); } else { throw e; } @@ -90,4 +83,17 @@ public class S3Utils } } } + + private static void awaitNextRetry(Exception e, int nTry) throws InterruptedException + { + final long baseSleepMillis = 1000; + final double fuzziness = 0.2; + final long sleepMillis = Math.max( + baseSleepMillis, + (long) (baseSleepMillis * Math.pow(2, nTry) * + (1 + new Random().nextGaussian() * fuzziness)) + ); + log.info(e, "S3 fail on try %d, retrying in %,dms.", nTry, sleepMillis); + Thread.sleep(sleepMillis); + } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java index 59100b5fdeb..75d212886da 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -112,7 +112,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller } catch (IOException e) { FileUtils.deleteDirectory(outDir); - throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); + throw new IOException(String.format("Problem decompressing object[%s]", s3Obj), e); } finally { Closeables.closeQuietly(in); From 49a80572b7bc377e366275f95a8eabb524745dfc Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 13:24:43 -0700 Subject: [PATCH 14/68] fix master speed issue with immutable copy of in druid server, better exception handling in moving and dropping segments --- .../com/metamx/druid/client/DruidServer.java | 4 +- .../com/metamx/druid/master/DruidMaster.java | 205 +++++++----------- 2 files changed, 87 insertions(+), 122 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index ebd078e9383..e86efdad816 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.DruidServerMetadata; +import java.util.Collections; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -116,7 +117,8 @@ public class DruidServer implements Comparable @JsonProperty public Map getSegments() { - return ImmutableMap.copyOf(segments); + // Copying the map slows things down a lot here, don't use Immutable Map here + return Collections.unmodifiableMap(segments); } public DataSegment getSegment(String segmentName) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index c7c39d3fdd3..e0f7b5841ae 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -28,6 +28,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.Closeables; +import com.metamx.common.IAE; import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -247,144 +248,106 @@ public class DruidMaster public void moveSegment(String from, String to, String segmentName, final LoadPeonCallback callback) { - final DruidServer fromServer = serverInventoryView.getInventoryValue(from); - if (fromServer == null) { - throw new IllegalArgumentException(String.format("Unable to find server [%s]", from)); - } + try { + final DruidServer fromServer = serverInventoryView.getInventoryValue(from); + if (fromServer == null) { + throw new IAE("Unable to find server [%s]", from); + } - final DruidServer toServer = serverInventoryView.getInventoryValue(to); - if (toServer == null) { - throw new IllegalArgumentException(String.format("Unable to find server [%s]", to)); - } + final DruidServer toServer = serverInventoryView.getInventoryValue(to); + if (toServer == null) { + throw new IAE("Unable to find server [%s]", to); + } - if (to.equalsIgnoreCase(from)) { - throw new IllegalArgumentException( - String.format("Redundant command to move segment [%s] from [%s] to [%s]", segmentName, from, to) + if (to.equalsIgnoreCase(from)) { + throw new IAE("Redundant command to move segment [%s] from [%s] to [%s]", segmentName, from, to); + } + + final DataSegment segment = fromServer.getSegment(segmentName); + if (segment == null) { + throw new IAE("Unable to find segment [%s] on server [%s]", segmentName, from); + } + + final LoadQueuePeon loadPeon = loadManagementPeons.get(to); + if (loadPeon == null) { + throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", to); + } + + final LoadQueuePeon dropPeon = loadManagementPeons.get(from); + if (dropPeon == null) { + throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", from); + } + + final ServerHolder toHolder = new ServerHolder(toServer, loadPeon); + if (toHolder.getAvailableSize() < segment.getSize()) { + throw new IAE( + "Not enough capacity on server [%s] for segment [%s]. Required: %,d, available: %,d.", + to, + segment, + segment.getSize(), + toHolder.getAvailableSize() + ); + } + + final String toLoadQueueSegPath = ZKPaths.makePath(ZKPaths.makePath(zkPaths.getLoadQueuePath(), to), segmentName); + final String toServedSegPath = ZKPaths.makePath( + ZKPaths.makePath(zkPaths.getServedSegmentsPath(), to), segmentName ); - } - final DataSegment segment = fromServer.getSegment(segmentName); - if (segment == null) { - throw new IllegalArgumentException( - String.format("Unable to find segment [%s] on server [%s]", segmentName, from) - ); - } - - final LoadQueuePeon loadPeon = loadManagementPeons.get(to); - if (loadPeon == null) { - throw new IllegalArgumentException(String.format("LoadQueuePeon hasn't been created yet for path [%s]", to)); - } - - final LoadQueuePeon dropPeon = loadManagementPeons.get(from); - if (dropPeon == null) { - throw new IllegalArgumentException(String.format("LoadQueuePeon hasn't been created yet for path [%s]", from)); - } - - final ServerHolder toHolder = new ServerHolder(toServer, loadPeon); - if (toHolder.getAvailableSize() < segment.getSize()) { - throw new IllegalArgumentException( - String.format( - "Not enough capacity on server [%s] for segment [%s]. Required: %,d, available: %,d.", - to, - segment, - segment.getSize(), - toHolder.getAvailableSize() - ) - ); - } - - final String toLoadQueueSegPath = ZKPaths.makePath(ZKPaths.makePath(zkPaths.getLoadQueuePath(), to), segmentName); - final String toServedSegPath = ZKPaths.makePath( - ZKPaths.makePath(zkPaths.getServedSegmentsPath(), to), segmentName - ); - - loadPeon.loadSegment( - segment, - new LoadPeonCallback() - { - @Override - protected void execute() + loadPeon.loadSegment( + segment, + new LoadPeonCallback() { - try { - if (curator.checkExists().forPath(toServedSegPath) != null && - curator.checkExists().forPath(toLoadQueueSegPath) == null && - !dropPeon.getSegmentsToDrop().contains(segment)) { - dropPeon.dropSegment(segment, callback); - } else if (callback != null) { - callback.execute(); + @Override + protected void execute() + { + try { + if (curator.checkExists().forPath(toServedSegPath) != null && + curator.checkExists().forPath(toLoadQueueSegPath) == null && + !dropPeon.getSegmentsToDrop().contains(segment)) { + dropPeon.dropSegment(segment, callback); + } else if (callback != null) { + callback.execute(); + } + } + catch (Exception e) { + throw Throwables.propagate(e); } } - catch (Exception e) { - throw Throwables.propagate(e); - } } - } - ); - } - - public void cloneSegment(String from, String to, String segmentName, LoadPeonCallback callback) - { - final DruidServer fromServer = serverInventoryView.getInventoryValue(from); - if (fromServer == null) { - throw new IllegalArgumentException(String.format("Unable to find server [%s]", from)); - } - - final DruidServer toServer = serverInventoryView.getInventoryValue(to); - if (toServer == null) { - throw new IllegalArgumentException(String.format("Unable to find server [%s]", to)); - } - - final DataSegment segment = fromServer.getSegment(segmentName); - if (segment == null) { - throw new IllegalArgumentException( - String.format("Unable to find segment [%s] on server [%s]", segmentName, from) ); } - - final LoadQueuePeon loadPeon = loadManagementPeons.get(to); - if (loadPeon == null) { - throw new IllegalArgumentException(String.format("LoadQueuePeon hasn't been created yet for path [%s]", to)); - } - - final ServerHolder toHolder = new ServerHolder(toServer, loadPeon); - if (toHolder.getAvailableSize() < segment.getSize()) { - throw new IllegalArgumentException( - String.format( - "Not enough capacity on server [%s] for segment [%s]. Required: %,d, available: %,d.", - to, - segment, - segment.getSize(), - toHolder.getAvailableSize() - ) - ); - } - - if (!loadPeon.getSegmentsToLoad().contains(segment)) { - loadPeon.loadSegment(segment, callback); + catch (Exception e) { + log.makeAlert(e, "Exception moving segment %s", segmentName).emit(); + callback.execute(); } } public void dropSegment(String from, String segmentName, final LoadPeonCallback callback) { - final DruidServer fromServer = serverInventoryView.getInventoryValue(from); - if (fromServer == null) { - throw new IllegalArgumentException(String.format("Unable to find server [%s]", from)); - } + try { + final DruidServer fromServer = serverInventoryView.getInventoryValue(from); + if (fromServer == null) { + throw new IAE("Unable to find server [%s]", from); + } - final DataSegment segment = fromServer.getSegment(segmentName); - if (segment == null) { - throw new IllegalArgumentException( - String.format("Unable to find segment [%s] on server [%s]", segmentName, from) - ); - } + final DataSegment segment = fromServer.getSegment(segmentName); + if (segment == null) { + throw new IAE("Unable to find segment [%s] on server [%s]", segmentName, from); + } - final LoadQueuePeon dropPeon = loadManagementPeons.get(from); - if (dropPeon == null) { - throw new IllegalArgumentException(String.format("LoadQueuePeon hasn't been created yet for path [%s]", from)); - } + final LoadQueuePeon dropPeon = loadManagementPeons.get(from); + if (dropPeon == null) { + throw new IAE("LoadQueuePeon hasn't been created yet for path [%s]", from); + } - if (!dropPeon.getSegmentsToDrop().contains(segment)) { - dropPeon.dropSegment(segment, callback); + if (!dropPeon.getSegmentsToDrop().contains(segment)) { + dropPeon.dropSegment(segment, callback); + } + } + catch (Exception e) { + log.makeAlert(e, "Exception dropping segment %s", segmentName).emit(); + callback.execute(); } } @@ -543,7 +506,7 @@ public class DruidMaster } catch (Exception e) { log.makeAlert(e, "Unable to become master") - .emit(); + .emit(); final LeaderLatch oldLatch = createNewLeaderLatch(); Closeables.closeQuietly(oldLatch); try { From 36756aa557d8b000a804f42b3bd1004812de355e Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 13:30:16 -0700 Subject: [PATCH 15/68] [maven-release-plugin] prepare release druid-0.5.6 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 5 ++--- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 12 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 6979f20000c..e5ad8f81835 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/common/pom.xml b/common/pom.xml index 1964f5658ff..6f8586370b3 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/examples/pom.xml b/examples/pom.xml index 9acca31f12d..73d14616b9c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples @@ -10,7 +9,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index fd727328c49..a94c6700c89 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index a8b8d429c79..fc7f24ba236 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 37cb5b9c2e4..343f982bf39 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/pom.xml b/pom.xml index dd042d33c33..4e6952364a8 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.6-SNAPSHOT + 0.5.6 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index abe217c1af2..403388cae4e 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/server/pom.xml b/server/pom.xml index e32b027219a..0fea0a499cb 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 diff --git a/services/pom.xml b/services/pom.xml index d01c1f44706..9197a633783 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.6-SNAPSHOT + 0.5.6 com.metamx druid - 0.5.6-SNAPSHOT + 0.5.6 From afac15d6310eb69ffd69ed5f38842836d7ff788d Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 13:32:50 -0700 Subject: [PATCH 16/68] fix version problem --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index e5ad8f81835..fd5039deddc 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 6f8586370b3..caea6c52f73 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 73d14616b9c..16d53467ae4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index a94c6700c89..86fdc42daef 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index fc7f24ba236..c3bea71fff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 343f982bf39..af06cc1cef5 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/pom.xml b/pom.xml index 16e717b9a2c..8bc94f28cb5 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.6 + 0.5.7-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 403388cae4e..3cf5f0dfcf7 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 0fea0a499cb..06d0f7795d6 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 9197a633783..c1bbb4bc3ec 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.6 + 0.5.7-SNAPSHOT com.metamx druid - 0.5.6 + 0.5.7-SNAPSHOT From cd174dbd8059607b83202c576a41c229bb67d4d6 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 13:35:29 -0700 Subject: [PATCH 17/68] [maven-release-plugin] prepare release druid-0.5.7 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index fd5039deddc..7e81d6a9544 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/common/pom.xml b/common/pom.xml index caea6c52f73..167a6b9f2f6 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/examples/pom.xml b/examples/pom.xml index 16d53467ae4..f7bc2ed8a73 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 86fdc42daef..6501ff65e5f 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index c3bea71fff5..31e67ca629e 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index af06cc1cef5..be1ef0ae77c 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/pom.xml b/pom.xml index 8bc94f28cb5..414e9b861ff 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.7-SNAPSHOT + 0.5.7 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 3cf5f0dfcf7..a327dc6e014 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/server/pom.xml b/server/pom.xml index 06d0f7795d6..1d5f8f00761 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 diff --git a/services/pom.xml b/services/pom.xml index c1bbb4bc3ec..26d12cea553 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.7-SNAPSHOT + 0.5.7 com.metamx druid - 0.5.7-SNAPSHOT + 0.5.7 From f422bfae8c352b39bf0632239bb17aa09559c20e Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 13:35:38 -0700 Subject: [PATCH 18/68] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7e81d6a9544..d412317b975 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 167a6b9f2f6..4f6dd9c1f8a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index f7bc2ed8a73..bf57f056f5e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 6501ff65e5f..475b202c5c1 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 31e67ca629e..ff27977ee94 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index be1ef0ae77c..3a47343a8b1 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/pom.xml b/pom.xml index 414e9b861ff..dc5076e747e 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.7 + 0.5.8-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a327dc6e014..e8ea64ef101 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 1d5f8f00761..6431dcda254 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 26d12cea553..febfc6462f4 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.7 + 0.5.8-SNAPSHOT com.metamx druid - 0.5.7 + 0.5.8-SNAPSHOT From 4e84fb5f036366f1770980ae504a223fb251d01b Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 16:50:52 -0700 Subject: [PATCH 19/68] better logic for exiting load rule assign if replicants cannot be created --- .../metamx/druid/master/rules/LoadRule.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index 9677b5028bc..3c6c58593e7 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -92,6 +92,12 @@ public abstract class LoadRule implements Rule final MasterStats stats = new MasterStats(); while (totalReplicants < expectedReplicants) { + boolean replicate = totalReplicants > 0; + + if (replicate && !replicationManager.canAddReplicant(getTier())) { + break; + } + final ServerHolder holder = analyzer.findNewSegmentHomeAssign(segment, serverHolderList); if (holder == null) { @@ -104,15 +110,10 @@ public abstract class LoadRule implements Rule break; } - if (totalReplicants > 0) { // don't throttle if there's only 1 copy of this segment in the cluster - if (!replicationManager.canAddReplicant(getTier()) || - !replicationManager.registerReplicantCreation( - getTier(), - segment.getIdentifier(), - holder.getServer().getHost() - )) { - break; - } + if (replicate && !replicationManager.registerReplicantCreation( + getTier(), segment.getIdentifier(), holder.getServer().getHost() + )) { + break; } holder.getPeon().loadSegment( From a3879d11518b60e10e8d31eef6abe98e1baf6fa8 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 16:53:41 -0700 Subject: [PATCH 20/68] [maven-release-plugin] prepare release druid-0.5.8 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index d412317b975..2e295d7004e 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/common/pom.xml b/common/pom.xml index 4f6dd9c1f8a..e3c33fd7c95 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/examples/pom.xml b/examples/pom.xml index bf57f056f5e..628bbee6288 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 475b202c5c1..b8b7500e323 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index ff27977ee94..ade2a10b23b 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 3a47343a8b1..65c2364a891 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/pom.xml b/pom.xml index dc5076e747e..e040af42421 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.8-SNAPSHOT + 0.5.8 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index e8ea64ef101..8e2c3cf3dfe 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/server/pom.xml b/server/pom.xml index 6431dcda254..a5e99db8f53 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 diff --git a/services/pom.xml b/services/pom.xml index febfc6462f4..46c9ad2f781 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.8-SNAPSHOT + 0.5.8 com.metamx druid - 0.5.8-SNAPSHOT + 0.5.8 From 0f492e17c3f91c8acb485653d37e6f93150ffc97 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 16:53:51 -0700 Subject: [PATCH 21/68] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 2e295d7004e..08f2afb9e0b 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index e3c33fd7c95..14bcb075fa0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 628bbee6288..e4b70f9730c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index b8b7500e323..7bdd9dc0964 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index ade2a10b23b..5d8a6be1b8d 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 65c2364a891..a6164570d53 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/pom.xml b/pom.xml index e040af42421..e7015ef0cad 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.8 + 0.5.9-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8e2c3cf3dfe..38e49dd792f 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index a5e99db8f53..df34ded8d95 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 46c9ad2f781..9e0ac6d6540 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.8 + 0.5.9-SNAPSHOT com.metamx druid - 0.5.8 + 0.5.9-SNAPSHOT From eb89857bdc76b4237b2320f6c7b06225da7e96a1 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 17:31:55 -0700 Subject: [PATCH 22/68] [maven-release-plugin] prepare release druid-0.5.9 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 08f2afb9e0b..d77bb9fda6f 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/common/pom.xml b/common/pom.xml index 14bcb075fa0..3f76addb32c 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/examples/pom.xml b/examples/pom.xml index e4b70f9730c..3aebae1e9dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 7bdd9dc0964..06b9fe0f82b 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 5d8a6be1b8d..ef43172b29e 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a6164570d53..253ec255b4c 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/pom.xml b/pom.xml index e7015ef0cad..dcf8a4327e8 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.9-SNAPSHOT + 0.5.9 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 38e49dd792f..e33dc78a9df 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/server/pom.xml b/server/pom.xml index df34ded8d95..cfd49769fc0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 diff --git a/services/pom.xml b/services/pom.xml index 9e0ac6d6540..3306f591d43 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.9-SNAPSHOT + 0.5.9 com.metamx druid - 0.5.9-SNAPSHOT + 0.5.9 From f3187b67a50e462ad173de8f0e6da556cb5fa3ac Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 17:32:05 -0700 Subject: [PATCH 23/68] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index d77bb9fda6f..fc34d7cc05e 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 3f76addb32c..f41dbca44f6 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 3aebae1e9dd..a9d0ab36cbb 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 06b9fe0f82b..a2dd70725ca 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index ef43172b29e..5fc8e8055e9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 253ec255b4c..cd2a0ac55cc 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/pom.xml b/pom.xml index dcf8a4327e8..f266cdde4a9 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.9 + 0.5.10-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index e33dc78a9df..128b7a21f67 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index cfd49769fc0..6ad80b716cf 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 3306f591d43..bfbf7040890 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.9 + 0.5.10-SNAPSHOT com.metamx druid - 0.5.9 + 0.5.10-SNAPSHOT From d5fee45bf4b396979bd44e45743408f40df4af3d Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 17:55:29 -0700 Subject: [PATCH 24/68] [maven-release-plugin] prepare release druid-0.5.10 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index fc34d7cc05e..28bbfcb3a84 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/common/pom.xml b/common/pom.xml index f41dbca44f6..99388944c45 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/examples/pom.xml b/examples/pom.xml index a9d0ab36cbb..2c33e550634 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index a2dd70725ca..51f790dc5f9 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 5fc8e8055e9..5fe92293099 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index cd2a0ac55cc..4d9ca3ab169 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/pom.xml b/pom.xml index f266cdde4a9..7bad48c21cf 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.10-SNAPSHOT + 0.5.10 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 128b7a21f67..26ca1c9c43d 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/server/pom.xml b/server/pom.xml index 6ad80b716cf..053267d64dd 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 diff --git a/services/pom.xml b/services/pom.xml index bfbf7040890..afc69af99fb 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.10-SNAPSHOT + 0.5.10 com.metamx druid - 0.5.10-SNAPSHOT + 0.5.10 From 43bdb99593345f46e51528b6815e84bdbee9b96d Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 16 Jul 2013 17:55:42 -0700 Subject: [PATCH 25/68] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 28bbfcb3a84..26fc807cb17 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 99388944c45..7baddd5eade 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 2c33e550634..9d4ccef3b11 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 51f790dc5f9..4d915c2daaf 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 5fe92293099..2e308648d6f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4d9ca3ab169..b40234655ba 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/pom.xml b/pom.xml index 7bad48c21cf..09e2976c593 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.10 + 0.5.11-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 26ca1c9c43d..7af1473640d 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 053267d64dd..287e0ae6bb7 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index afc69af99fb..02e420abab8 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.10 + 0.5.11-SNAPSHOT com.metamx druid - 0.5.10 + 0.5.11-SNAPSHOT From 22526a6c68f238c6d125a3210ee42c2a6530c234 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 17 Jul 2013 09:14:28 -0700 Subject: [PATCH 26/68] [maven-release-plugin] prepare release druid-0.5.11 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 26fc807cb17..f4cc1883521 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/common/pom.xml b/common/pom.xml index 7baddd5eade..ff08a619f49 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/examples/pom.xml b/examples/pom.xml index 9d4ccef3b11..2be3a5e9356 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 4d915c2daaf..4b63cbebde4 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 2e308648d6f..a05d73c5b43 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index b40234655ba..a6e0246f2b1 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/pom.xml b/pom.xml index 09e2976c593..044f4626c35 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.11-SNAPSHOT + 0.5.11 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 7af1473640d..656d8a4c5a6 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/server/pom.xml b/server/pom.xml index 287e0ae6bb7..45b2a5eb129 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 diff --git a/services/pom.xml b/services/pom.xml index 02e420abab8..aa1ecbeba71 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.11-SNAPSHOT + 0.5.11 com.metamx druid - 0.5.11-SNAPSHOT + 0.5.11 From cef0e508811ff90db84f33961eb60d1dd067c60c Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 17 Jul 2013 09:14:37 -0700 Subject: [PATCH 27/68] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index f4cc1883521..68b5e99d8a4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index ff08a619f49..c48c0fe20c8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 2be3a5e9356..39e52cf5b51 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 4b63cbebde4..fd9eba8f451 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index a05d73c5b43..4f130f14fa8 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a6e0246f2b1..d290229eb8e 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/pom.xml b/pom.xml index 044f4626c35..9c3cc0f3231 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.11 + 0.5.12-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 656d8a4c5a6..9e8c5d96983 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 45b2a5eb129..f5aa2785d8e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index aa1ecbeba71..9cc6497f2d2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.11 + 0.5.12-SNAPSHOT com.metamx druid - 0.5.11 + 0.5.12-SNAPSHOT From d4afeb5ebe3fff3d6c22350b4b46435e1256af98 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 18 Jul 2013 11:18:28 -0700 Subject: [PATCH 28/68] 1) upload.sh pushes up to artifacts/releases 2) Delete some commented stuff in build.sh --- build.sh | 2 -- upload.sh | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/build.sh b/build.sh index fe534e319f7..158a0cd6153 100755 --- a/build.sh +++ b/build.sh @@ -10,8 +10,6 @@ SCRIPT_DIR=`pwd` popd VERSION=`cat pom.xml | grep version | head -4 | tail -1 | sed 's_.*\([^<]*\).*_\1_'` -#TAR_FILE=${SCRIPT_DIR}/${PROJECT}-${VERSION}.tar.gz -#rm -f ${TAR_FILE} echo Using Version[${VERSION}] diff --git a/upload.sh b/upload.sh index 5e0580d3055..d242599adac 100755 --- a/upload.sh +++ b/upload.sh @@ -3,4 +3,4 @@ # # Script to upload tarball of assembly build to static.druid.io for serving # -s3cmd put services/target/druid-services-*-bin.tar.gz s3://static.druid.io/artifacts/ +s3cmd put services/target/druid-services-*-bin.tar.gz s3://static.druid.io/artifacts/releases From da658ac69a0588e2588ab21b9f891152e0d387e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Thu, 18 Jul 2013 19:30:14 +0000 Subject: [PATCH 29/68] Changes according to Eric's comments on the pull request. - moved the RabbitMQ ConnectionFactory configuration to a method of its own to clean up the FirehoseFactory.connect() method. - using Throwables.propagate*() methods in exception handling. - removed TODOs and instead using the "keyword" FUTURE as well as adding some context to the comments themselves. - cleaned up the exception handling in hasMore() a little and made it more readable. --- .../firehose/RabbitMQFirehoseFactory.java | 42 +++++++++++-------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java index 36dbad4d106..ed5856e9679 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/RabbitMQFirehoseFactory.java @@ -2,6 +2,7 @@ package com.metamx.druid.realtime.firehose; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.input.InputRow; @@ -75,10 +76,8 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory this.parser = parser; } - @Override - public Firehose connect() throws IOException + private void primeConnectionFactory(ConnectionFactory factory) throws IOException { - final ConnectionFactory factory = new ConnectionFactory(); factory.setHost(consumerProps.getProperty("host", factory.getHost())); factory.setPort(Integer.parseInt(consumerProps.getProperty("port", Integer.toString(factory.getPort())))); factory.setUsername(consumerProps.getProperty("username", factory.getUsername())); @@ -91,10 +90,17 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory factory.setUri(consumerProps.getProperty("uri")); } catch(Exception e){ - // A little silly to throw an IOException but we'll make do for now with it. - throw new IOException("Bad URI format.", e); + Throwables.propagateIfPossible(e, IOException.class); + throw Throwables.propagate(e); } } + } + + @Override + public Firehose connect() throws IOException + { + final ConnectionFactory factory = new ConnectionFactory(); + primeConnectionFactory(factory); String queue = consumerProps.getProperty("queue"); String exchange = consumerProps.getProperty("exchange"); @@ -111,7 +117,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory public void shutdownCompleted(ShutdownSignalException cause) { log.warn(cause, "Connection closed!"); - //TODO: should we re-establish the connection here? + //FUTURE: we could try to re-establish the connection here. Not done in this version though. } }); @@ -124,7 +130,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory public void shutdownCompleted(ShutdownSignalException cause) { log.warn(cause, "Channel closed!"); - //TODO: should we re-establish the connection here? + //FUTURE: we could try to re-establish the connection here. Not done in this version though. } }); @@ -152,23 +158,23 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory { delivery = null; try { + // Wait for the next delivery. This will block until something is available. delivery = consumer.nextDelivery(); - lastDeliveryTag = delivery.getEnvelope().getDeliveryTag(); - //log.debug("Received new message from RabbitMQ"); + if (delivery != null) { + lastDeliveryTag = delivery.getEnvelope().getDeliveryTag(); + // If delivery is non-null, we report that there is something more to process. + return true; + } } catch (InterruptedException e) { - //TODO: Not exactly sure how we should react to this. - // Does it mean that delivery will be null and we should handle that - // as if there are no more messages (return false)? + // A little unclear on how we should handle this. + + // At any rate, we're in an unknown state now so let's log something and return false. log.wtf(e, "Got interrupted while waiting for next delivery. Doubt this should ever happen."); } - if (delivery != null) { - // If delivery is non-null, we report that there is something more to process. - return true; - } - - // This means that delivery is null so we have nothing more to process. + // This means that delivery is null or we caught the exception above so we report that we have + // nothing more to process. return false; } From ae4132adba0714e8cad2ea4ceeab7eeb4bf700fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Thu, 18 Jul 2013 19:42:07 +0000 Subject: [PATCH 30/68] Adding a test producer application. --- .../rabbitmq/RabbitMQProducerMain.java | 182 ++++++++++++++++++ 1 file changed, 182 insertions(+) create mode 100644 examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java diff --git a/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java b/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java new file mode 100644 index 00000000000..a070bfff929 --- /dev/null +++ b/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java @@ -0,0 +1,182 @@ +package druid.examples.rabbitmq; + +import com.rabbitmq.client.Channel; +import com.rabbitmq.client.Connection; +import com.rabbitmq.client.ConnectionFactory; +import org.apache.commons.cli.*; + +import java.text.SimpleDateFormat; +import java.util.*; + +/** + * + */ +public class RabbitMQProducerMain +{ + public static void main(String[] args) + throws Exception + { + // We use a List to keep track of option insertion order. See below. + final List

  *   "firehose" : {
  *     "type" : "rabbitmq",
- *     "consumerProps" : {
- *       "host": "localhost",                 # The hostname of the RabbitMQ broker to connect to. Default: 'localhost'
- *       "port": "5672",                      # The port number to connect to on the RabbitMQ broker. Default: '5672'
- *       "username": "test-dude",             # The username to use to connect to RabbitMQ. Default: 'guest'
- *       "password": "test-word",             # The password to use to connect to RabbitMQ. Default: 'guest'
- *       "virtualHost": "test-vhost",         # The virtual host to connect to. Default: '/'
- *       "uri": "amqp://mqserver:1234/vhost", # The URI string to use to connect to RabbitMQ. No default and not needed
- *       "exchange": "test-exchange",         # The exchange to connect to. No default
- *       "queue" : "druidtest",               # The queue to connect to or create. No default
- *       "routingKey": "#",                   # The routing key to use to bind the queue to the exchange. No default
- *       "durable": "true",                   # Whether the queue should be durable. Default: 'false'
- *       "exclusive": "false",                # Whether the queue should be exclusive. Default: 'false'
- *       "autoDelete": "false"                # Whether the queue should auto-delete on disconnect. Default: 'false'
- *     },
+ *     "connection" : {
+  *       "host": "localhost",                 # The hostname of the RabbitMQ broker to connect to. Default: 'localhost'
+  *       "port": "5672",                      # The port number to connect to on the RabbitMQ broker. Default: '5672'
+  *       "username": "test-dude",             # The username to use to connect to RabbitMQ. Default: 'guest'
+  *       "password": "test-word",             # The password to use to connect to RabbitMQ. Default: 'guest'
+  *       "virtualHost": "test-vhost",         # The virtual host to connect to. Default: '/'
+  *       "uri": "amqp://mqserver:1234/vhost", # The URI string to use to connect to RabbitMQ. No default and not needed
+  *     },
+  *     "config" : {
+  *       "exchange": "test-exchange",         # The exchange to connect to. No default
+  *       "queue" : "druidtest",               # The queue to connect to or create. No default
+  *       "routingKey": "#",                   # The routing key to use to bind the queue to the exchange. No default
+  *       "durable": "true",                   # Whether the queue should be durable. Default: 'false'
+  *       "exclusive": "false",                # Whether the queue should be exclusive. Default: 'false'
+  *       "autoDelete": "false"                # Whether the queue should auto-delete on disconnect. Default: 'false'
+  *     },
  *     "parser" : {
  *       "timestampSpec" : { "column" : "utcdt", "format" : "iso" },
  *       "data" : { "format" : "json" },

From 15f756c3a4e58c36ca4fbae006b8d07724f74623 Mon Sep 17 00:00:00 2001
From: cheddar 
Date: Fri, 19 Jul 2013 16:57:55 -0700
Subject: [PATCH 51/68] Fix upload script to actually upload into the directory

---
 upload.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/upload.sh b/upload.sh
index d242599adac..430d450122a 100755
--- a/upload.sh
+++ b/upload.sh
@@ -3,4 +3,4 @@
 #
 # Script to upload tarball of assembly build to static.druid.io for serving
 #
-s3cmd put services/target/druid-services-*-bin.tar.gz s3://static.druid.io/artifacts/releases
+s3cmd put services/target/druid-services-*-bin.tar.gz s3://static.druid.io/artifacts/releases/

From 2d0fae2fdeef1c03ca7ff638d73f5d15b68c9b16 Mon Sep 17 00:00:00 2001
From: fjy 
Date: Mon, 22 Jul 2013 10:47:11 -0700
Subject: [PATCH 52/68] add to string to spatial wrapper

---
 .../druid/index/v1/SpatialDimensionRowFormatter.java   | 10 ++++++++++
 1 file changed, 10 insertions(+)

diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java
index f5dad643ad5..b807f931c5d 100644
--- a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java
+++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Sets;
 import com.google.common.primitives.Floats;
 import com.metamx.common.ISE;
 import com.metamx.druid.input.InputRow;
+import org.joda.time.DateTime;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
@@ -138,6 +139,15 @@ public class SpatialDimensionRowFormatter
       {
         return row.getFloatMetric(metric);
       }
+
+      @Override
+      public String toString()
+      {
+        return "InputRow{" +
+               "timestamp=" + row.getTimestampFromEpoch() +
+               ", dimensions=" + row.getDimensions() +
+               '}';
+      }
     };
 
     if (!spatialPartialDimNames.isEmpty()) {

From f5536ed010752bcadccda3c568c2d065cdaacfed Mon Sep 17 00:00:00 2001
From: fjy 
Date: Mon, 22 Jul 2013 14:05:48 -0700
Subject: [PATCH 53/68] compression for batch-zk

---
 .../java/com/metamx/druid/QueryableNode.java  | 80 +++++++++++++------
 .../druid/initialization/CuratorConfig.java   |  4 +
 .../druid/initialization/Initialization.java  | 20 ++---
 .../ZkDataSegmentAnnouncerConfig.java         |  4 +
 .../v1/SpatialDimensionRowFormatter.java      |  5 +-
 5 files changed, 74 insertions(+), 39 deletions(-)

diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java
index c715de91f97..f324d4e7e9f 100644
--- a/client/src/main/java/com/metamx/druid/QueryableNode.java
+++ b/client/src/main/java/com/metamx/druid/QueryableNode.java
@@ -373,18 +373,21 @@ public abstract class QueryableNode extends Registering
     if (requestLogger == null) {
       try {
         final String loggingType = props.getProperty("druid.request.logging.type");
-        if("emitter".equals(loggingType)) {
-          setRequestLogger(Initialization.makeEmittingRequestLogger(
-            getProps(),
-            getEmitter()
-          ));
-        }
-        else if ("file".equalsIgnoreCase(loggingType)) {
-          setRequestLogger(Initialization.makeFileRequestLogger(
-            getJsonMapper(),
-            getScheduledExecutorFactory(),
-            getProps()
-          ));
+        if ("emitter".equals(loggingType)) {
+          setRequestLogger(
+              Initialization.makeEmittingRequestLogger(
+                  getProps(),
+                  getEmitter()
+              )
+          );
+        } else if ("file".equalsIgnoreCase(loggingType)) {
+          setRequestLogger(
+              Initialization.makeFileRequestLogger(
+                  getJsonMapper(),
+                  getScheduledExecutorFactory(),
+                  getProps()
+              )
+          );
         } else {
           setRequestLogger(new NoopRequestLogger());
         }
@@ -428,19 +431,46 @@ public abstract class QueryableNode extends Registering
       final Announcer announcer = new Announcer(getCuratorFramework(), Execs.singleThreaded("Announcer-%s"));
       lifecycle.addManagedInstance(announcer);
 
-      setAnnouncer(
-          new MultipleDataSegmentAnnouncerDataSegmentAnnouncer(
-              Arrays.asList(
-                  new BatchingCuratorDataSegmentAnnouncer(
-                      getDruidServerMetadata(),
-                      getConfigFactory().build(ZkDataSegmentAnnouncerConfig.class),
-                      announcer,
-                      getJsonMapper()
-                  ),
-                  new CuratorDataSegmentAnnouncer(getDruidServerMetadata(), getZkPaths(), announcer, getJsonMapper())
-              )
-          )
-      );
+      final ZkDataSegmentAnnouncerConfig config = getConfigFactory().build(ZkDataSegmentAnnouncerConfig.class);
+      final String announcerType = config.getAnnouncerType();
+
+      final DataSegmentAnnouncer dataSegmentAnnouncer;
+      if ("curator".equalsIgnoreCase(announcerType)) {
+        dataSegmentAnnouncer = new CuratorDataSegmentAnnouncer(
+            getDruidServerMetadata(),
+            getZkPaths(),
+            announcer,
+            getJsonMapper()
+        );
+      } else if ("batch".equalsIgnoreCase(announcerType)) {
+        dataSegmentAnnouncer = new BatchingCuratorDataSegmentAnnouncer(
+            getDruidServerMetadata(),
+            config,
+            announcer,
+            getJsonMapper()
+        );
+      } else if ("multiple".equalsIgnoreCase(announcerType)) {
+        dataSegmentAnnouncer = new MultipleDataSegmentAnnouncerDataSegmentAnnouncer(
+            Arrays.asList(
+                new BatchingCuratorDataSegmentAnnouncer(
+                    getDruidServerMetadata(),
+                    config,
+                    announcer,
+                    getJsonMapper()
+                ),
+                new CuratorDataSegmentAnnouncer(
+                    getDruidServerMetadata(),
+                    getZkPaths(),
+                    announcer,
+                    getJsonMapper()
+                )
+            )
+        );
+      } else {
+        throw new ISE("Unknown announcer type [%s]", announcerType);
+      }
+
+      setAnnouncer(dataSegmentAnnouncer);
 
       lifecycle.addManagedInstance(getAnnouncer(), Lifecycle.Stage.LAST);
     }
diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java
index d09c36f369e..741bc59d3d9 100644
--- a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java
+++ b/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java
@@ -32,4 +32,8 @@ public abstract class CuratorConfig
   @Config("druid.zk.service.sessionTimeoutMs")
   @Default("30000")
   public abstract int getZkSessionTimeoutMs();
+
+  @Config("druid.curator.compression.enable")
+  @Default("false")
+  public abstract boolean enableCompression();
 }
diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java
index 0d49ff0cefe..cd0f1147cda 100644
--- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java
+++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java
@@ -72,13 +72,13 @@ public class Initialization
   /**
    * Load properties.
    * Properties are layered:
-   *
+   * 

* # stored in zookeeper * # runtime.properties file, * # cmdLine -D - * + *

* command line overrides runtime.properties which overrides zookeeper - * + *

* Idempotent. Thread-safe. Properties are only loaded once. * If property druid.zk.service.host is not set then do not load properties from zookeeper. * @@ -196,10 +196,10 @@ public class Initialization CuratorFrameworkFactory.builder() .connectString(curatorConfig.getZkHosts()) .sessionTimeoutMs(curatorConfig.getZkSessionTimeoutMs()) - .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) - // Don't compress stuff written just yet, need to get code deployed first. - .compressionProvider(new PotentiallyGzippedCompressionProvider(false)) - .build(); + .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) + // Don't compress stuff written just yet, need to get code deployed first. + .compressionProvider(new PotentiallyGzippedCompressionProvider(curatorConfig.enableCompression())) + .build(); lifecycle.addHandler( new Lifecycle.Handler() @@ -335,9 +335,9 @@ public class Initialization } public static RequestLogger makeFileRequestLogger( - ObjectMapper objectMapper, - ScheduledExecutorFactory factory, - Properties props + ObjectMapper objectMapper, + ScheduledExecutorFactory factory, + Properties props ) throws IOException { return new FileRequestLogger( diff --git a/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java b/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java index 2ff9f9172ca..b35a1e0351c 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java @@ -14,4 +14,8 @@ public abstract class ZkDataSegmentAnnouncerConfig extends ZkPathsConfig @Config("druid.zk.maxNumBytesPerNode") @Default("512000") public abstract long getMaxNumBytes(); + + @Config("druid.announcer.type") + @Default("curator") + public abstract String getAnnouncerType(); } diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java index b807f931c5d..864a0087831 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java @@ -143,10 +143,7 @@ public class SpatialDimensionRowFormatter @Override public String toString() { - return "InputRow{" + - "timestamp=" + row.getTimestampFromEpoch() + - ", dimensions=" + row.getDimensions() + - '}'; + return row.toString(); } }; From b6aeb5b3762e4b0b7d1c572acde107e18ac1d950 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 22 Jul 2013 14:14:21 -0700 Subject: [PATCH 54/68] code review fixes --- .../src/main/java/com/metamx/druid/QueryableNode.java | 11 ++--------- .../initialization/ZkDataSegmentAnnouncerConfig.java | 2 +- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index f324d4e7e9f..c2952a1ba41 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -435,21 +435,14 @@ public abstract class QueryableNode extends Registering final String announcerType = config.getAnnouncerType(); final DataSegmentAnnouncer dataSegmentAnnouncer; - if ("curator".equalsIgnoreCase(announcerType)) { - dataSegmentAnnouncer = new CuratorDataSegmentAnnouncer( - getDruidServerMetadata(), - getZkPaths(), - announcer, - getJsonMapper() - ); - } else if ("batch".equalsIgnoreCase(announcerType)) { + if ("batch".equalsIgnoreCase(announcerType)) { dataSegmentAnnouncer = new BatchingCuratorDataSegmentAnnouncer( getDruidServerMetadata(), config, announcer, getJsonMapper() ); - } else if ("multiple".equalsIgnoreCase(announcerType)) { + } else if ("legacy".equalsIgnoreCase(announcerType)) { dataSegmentAnnouncer = new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( Arrays.asList( new BatchingCuratorDataSegmentAnnouncer( diff --git a/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java b/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java index b35a1e0351c..e14c65027eb 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java @@ -16,6 +16,6 @@ public abstract class ZkDataSegmentAnnouncerConfig extends ZkPathsConfig public abstract long getMaxNumBytes(); @Config("druid.announcer.type") - @Default("curator") + @Default("legacy") public abstract String getAnnouncerType(); } From 7504f3ab65977614a02c412ef388cc4a5603b895 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 22 Jul 2013 16:31:01 -0700 Subject: [PATCH 55/68] 1) Adjust the Announcer to cleanup parent nodes that it creates. This stops it from leaving little turdlings lying around on zookeeper as nodes exit 2) Rename *CuratorDataSegmentAnnouncer because neither of them depend directly on Curator. --- .../java/com/metamx/druid/QueryableNode.java | 8 +-- .../AbstractDataSegmentAnnouncer.java | 2 +- ...java => BatchingDataSegmentAnnouncer.java} | 13 +---- ...r.java => SingleDataSegmentAnnouncer.java} | 6 +- .../druid/curator/announcement/Announcer.java | 58 ++++++++++++++++--- ... => BatchingDataSegmentAnnouncerTest.java} | 6 +- .../curator/announcement/AnnouncerTest.java | 53 ++++++++++++++++- .../druid/coordination/ZkCoordinatorTest.java | 2 +- 8 files changed, 116 insertions(+), 32 deletions(-) rename client/src/main/java/com/metamx/druid/coordination/{BatchingCuratorDataSegmentAnnouncer.java => BatchingDataSegmentAnnouncer.java} (94%) rename client/src/main/java/com/metamx/druid/coordination/{CuratorDataSegmentAnnouncer.java => SingleDataSegmentAnnouncer.java} (93%) rename client/src/test/java/com/metamx/druid/coordination/{BatchingCuratorDataSegmentAnnouncerTest.java => BatchingDataSegmentAnnouncerTest.java} (97%) diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index c715de91f97..3236d1aa336 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -40,11 +40,11 @@ import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.ServerView; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.coordination.AbstractDataSegmentAnnouncer; -import com.metamx.druid.coordination.BatchingCuratorDataSegmentAnnouncer; -import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer; +import com.metamx.druid.coordination.BatchingDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.coordination.MultipleDataSegmentAnnouncerDataSegmentAnnouncer; +import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.http.NoopRequestLogger; import com.metamx.druid.http.RequestLogger; @@ -431,13 +431,13 @@ public abstract class QueryableNode extends Registering setAnnouncer( new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( Arrays.asList( - new BatchingCuratorDataSegmentAnnouncer( + new BatchingDataSegmentAnnouncer( getDruidServerMetadata(), getConfigFactory().build(ZkDataSegmentAnnouncerConfig.class), announcer, getJsonMapper() ), - new CuratorDataSegmentAnnouncer(getDruidServerMetadata(), getZkPaths(), announcer, getJsonMapper()) + new SingleDataSegmentAnnouncer(getDruidServerMetadata(), getZkPaths(), announcer, getJsonMapper()) ) ) ); diff --git a/client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java index 583910e3dac..1bb0b8114ee 100644 --- a/client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java @@ -86,7 +86,7 @@ public abstract class AbstractDataSegmentAnnouncer implements DataSegmentAnnounc return; } - log.info("Stopping CuratorDataSegmentAnnouncer with config[%s]", config); + log.info("Stopping %s with config[%s]", getClass(), config); announcer.unannounce(makeAnnouncementPath()); started = false; diff --git a/client/src/main/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/BatchingDataSegmentAnnouncer.java similarity index 94% rename from client/src/main/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncer.java rename to client/src/main/java/com/metamx/druid/coordination/BatchingDataSegmentAnnouncer.java index e8070e71ff7..6b098d41b7f 100644 --- a/client/src/main/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/BatchingDataSegmentAnnouncer.java @@ -22,33 +22,26 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; import org.joda.time.DateTime; import java.io.IOException; -import java.util.Comparator; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; /** */ -public class BatchingCuratorDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer +public class BatchingDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer { - private static final Logger log = new Logger(BatchingCuratorDataSegmentAnnouncer.class); + private static final Logger log = new Logger(BatchingDataSegmentAnnouncer.class); private final ZkDataSegmentAnnouncerConfig config; private final Announcer announcer; @@ -58,7 +51,7 @@ public class BatchingCuratorDataSegmentAnnouncer extends AbstractDataSegmentAnno private final Set availableZNodes = Sets.newHashSet(); private final Map segmentLookup = Maps.newHashMap(); - public BatchingCuratorDataSegmentAnnouncer( + public BatchingDataSegmentAnnouncer( DruidServerMetadata server, ZkDataSegmentAnnouncerConfig config, Announcer announcer, diff --git a/client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java similarity index 93% rename from client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java rename to client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java index f9286d1a239..1757ccbe6af 100644 --- a/client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java @@ -28,15 +28,15 @@ import org.apache.curator.utils.ZKPaths; import java.io.IOException; -public class CuratorDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer +public class SingleDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer { - private static final Logger log = new Logger(CuratorDataSegmentAnnouncer.class); + private static final Logger log = new Logger(SingleDataSegmentAnnouncer.class); private final Announcer announcer; private final ObjectMapper jsonMapper; private final String servedSegmentsLocation; - public CuratorDataSegmentAnnouncer( + public SingleDataSegmentAnnouncer( DruidServerMetadata server, ZkPathsConfig config, Announcer announcer, diff --git a/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java b/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java index 57981f91785..727f7704771 100644 --- a/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java +++ b/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java @@ -48,6 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; @@ -64,6 +65,7 @@ public class Announcer private final List> toAnnounce = Lists.newArrayList(); private final ConcurrentMap listeners = new MapMaker().makeMap(); private final ConcurrentMap> announcements = new MapMaker().makeMap(); + private final List parentsIBuilt = new CopyOnWriteArrayList(); private boolean started = false; @@ -114,6 +116,15 @@ public class Announcer unannounce(ZKPaths.makePath(basePath, announcementPath)); } } + + for (String parent : parentsIBuilt) { + try { + curator.delete().forPath(parent); + } + catch (Exception e) { + log.info(e, "Unable to delete parent[%s], boooo.", parent); + } + } } } @@ -136,10 +147,19 @@ public class Announcer final ZKPaths.PathAndNode pathAndNode = ZKPaths.getPathAndNode(path); final String parentPath = pathAndNode.getPath(); + boolean buildParentPath = false; ConcurrentMap subPaths = announcements.get(parentPath); if (subPaths == null) { + try { + if (curator.checkExists().forPath(parentPath) == null) { + buildParentPath = true; + } + } + catch (Exception e) { + log.debug(e, "Problem checking if the parent existed, ignoring."); + } // I don't have a watcher on this path yet, create a Map and start watching. announcements.putIfAbsent(parentPath, new MapMaker().makeMap()); @@ -208,17 +228,15 @@ public class Announcer } ); - try { - synchronized (toAnnounce) { - if (started) { - cache.start(); - listeners.put(parentPath, cache); + synchronized (toAnnounce) { + if (started) { + if (buildParentPath) { + createPath(parentPath); } + startCache(cache); + listeners.put(parentPath, cache); } } - catch (Exception e) { - throw Throwables.propagate(e); - } } } @@ -261,7 +279,7 @@ public class Announcer throw new ISE("Cannot update a path[%s] that hasn't been announced!", path); } - synchronized (subPaths) { + synchronized (toAnnounce) { try { byte[] oldBytes = subPaths.get(nodePath); @@ -320,4 +338,26 @@ public class Announcer throw Throwables.propagate(e); } } + + private void startCache(PathChildrenCache cache) + { + try { + cache.start(); + } + catch (Exception e) { + Closeables.closeQuietly(cache); + throw Throwables.propagate(e); + } + } + + private void createPath(String parentPath) + { + try { + curator.create().creatingParentsIfNeeded().forPath(parentPath); + parentsIBuilt.add(parentPath); + } + catch (Exception e) { + log.info(e, "Problem creating parentPath[%s], someone else created it first?", parentPath); + } + } } diff --git a/client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java b/client/src/test/java/com/metamx/druid/coordination/BatchingDataSegmentAnnouncerTest.java similarity index 97% rename from client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java rename to client/src/test/java/com/metamx/druid/coordination/BatchingDataSegmentAnnouncerTest.java index d4390587268..72b32f26c98 100644 --- a/client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java +++ b/client/src/test/java/com/metamx/druid/coordination/BatchingDataSegmentAnnouncerTest.java @@ -47,7 +47,7 @@ import java.util.Set; /** */ -public class BatchingCuratorDataSegmentAnnouncerTest +public class BatchingDataSegmentAnnouncerTest { private static final String testBasePath = "/test"; private static final String testSegmentsPath = "/test/segments/id"; @@ -58,7 +58,7 @@ public class BatchingCuratorDataSegmentAnnouncerTest private ObjectMapper jsonMapper; private Announcer announcer; private SegmentReader segmentReader; - private BatchingCuratorDataSegmentAnnouncer segmentAnnouncer; + private BatchingDataSegmentAnnouncer segmentAnnouncer; private Set testSegments; @Before @@ -84,7 +84,7 @@ public class BatchingCuratorDataSegmentAnnouncerTest announcer.start(); segmentReader = new SegmentReader(cf, jsonMapper); - segmentAnnouncer = new BatchingCuratorDataSegmentAnnouncer( + segmentAnnouncer = new BatchingDataSegmentAnnouncer( new DruidServerMetadata( "id", "host", diff --git a/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java b/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java index 21c9ae2eee0..9f4276e164a 100644 --- a/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java +++ b/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java @@ -27,6 +27,8 @@ import org.apache.curator.framework.api.CuratorEvent; import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.api.CuratorListener; import org.apache.curator.test.KillSession; +import org.apache.curator.utils.ZKPaths; +import org.apache.zookeeper.data.Stat; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -60,7 +62,6 @@ public class AnnouncerTest extends CuratorTestBase public void testSanity() throws Exception { curator.start(); - curator.create().forPath("/somewhere"); Announcer announcer = new Announcer(curator, exec); final byte[] billy = "billy".getBytes(); @@ -163,4 +164,54 @@ public class AnnouncerTest extends CuratorTestBase announcer.stop(); } } + + @Test + public void testCleansUpItsLittleTurdlings() throws Exception + { + curator.start(); + Announcer announcer = new Announcer(curator, exec); + + final byte[] billy = "billy".getBytes(); + final String testPath = "/somewhere/test2"; + final String parent = ZKPaths.getPathAndNode(testPath).getPath(); + + announcer.start(); + + Assert.assertNull(curator.checkExists().forPath(parent)); + + announcer.announce(testPath, billy); + + Assert.assertNotNull(curator.checkExists().forPath(parent)); + + announcer.stop(); + + Assert.assertNull(curator.checkExists().forPath(parent)); + } + + @Test + public void testLeavesBehindTurdlingsThatAlreadyExisted() throws Exception + { + curator.start(); + Announcer announcer = new Announcer(curator, exec); + + final byte[] billy = "billy".getBytes(); + final String testPath = "/somewhere/test2"; + final String parent = ZKPaths.getPathAndNode(testPath).getPath(); + + curator.create().forPath(parent); + final Stat initialStat = curator.checkExists().forPath(parent); + + announcer.start(); + + Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid()); + + announcer.announce(testPath, billy); + + Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid()); + + announcer.stop(); + + Assert.assertEquals(initialStat.getMzxid(), curator.checkExists().forPath(parent).getMzxid()); + } + } diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index 1a34c75cdbc..746e7035cbb 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -93,7 +93,7 @@ public class ZkCoordinatorTest extends CuratorTestBase } }; - announcer = new CuratorDataSegmentAnnouncer( + announcer = new SingleDataSegmentAnnouncer( me, zkPaths, new Announcer(curator, Execs.singleThreaded("blah")), jsonMapper ); From 3f97ac92532d6a91946f612a782311ac19935f30 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 22 Jul 2013 18:01:13 -0700 Subject: [PATCH 56/68] add read to batch zk --- .../java/com/metamx/druid/QueryableNode.java | 36 +++- .../client/BatchingServerInventoryVIew.java | 108 +++++++++++ .../druid/client/ServerInventoryView.java | 179 +++++++++++------- .../client/ServerInventoryViewConfig.java | 4 + .../client/SingleServerInventoryView.java | 71 +++++++ .../inventory/CuratorInventoryManager.java | 13 +- .../CuratorInventoryManagerStrategy.java | 1 + ...tchingCuratorDataSegmentAnnouncerTest.java | 6 + .../CuratorInventoryManagerTest.java | 8 + .../com/metamx/druid/http/MasterMain.java | 31 ++- .../com/metamx/druid/master/DruidMaster.java | 2 +- .../metamx/druid/master/DruidMasterTest.java | 6 +- 12 files changed, 379 insertions(+), 86 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java create mode 100644 client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index c2952a1ba41..1741e13a27e 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -26,6 +26,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -33,11 +34,13 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.BatchingServerInventoryView; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.ServerView; +import com.metamx.druid.client.SingleServerInventoryView; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.coordination.AbstractDataSegmentAnnouncer; import com.metamx.druid.coordination.BatchingCuratorDataSegmentAnnouncer; @@ -343,7 +346,8 @@ public abstract class QueryableNode extends Registering } } - private void initializeInventoryView() + private void + initializeInventoryView() { if (inventoryView == null) { initializeServerInventoryView(); @@ -357,13 +361,29 @@ public abstract class QueryableNode extends Registering final ExecutorService exec = Executors.newFixedThreadPool( 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() ); - serverInventoryView = new ServerInventoryView( - getConfigFactory().build(ServerInventoryViewConfig.class), - getZkPaths(), - getCuratorFramework(), - exec, - getJsonMapper() - ); + + final ServerInventoryViewConfig serverInventoryViewConfig = getConfigFactory().build(ServerInventoryViewConfig.class); + final String announcerType = serverInventoryViewConfig.getAnnouncerType(); + + if ("legacy".equalsIgnoreCase(announcerType)) { + serverInventoryView = new SingleServerInventoryView( + serverInventoryViewConfig, + getZkPaths(), + getCuratorFramework(), + exec, + getJsonMapper() + ); + } else if ("batch".equalsIgnoreCase(announcerType)) { + serverInventoryView = new BatchingServerInventoryView( + serverInventoryViewConfig, + getZkPaths(), + getCuratorFramework(), + exec, + getJsonMapper() + ); + } else { + throw new IAE("Unknown type %s", announcerType); + } lifecycle.addManagedInstance(serverInventoryView); } } diff --git a/client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java b/client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java new file mode 100644 index 00000000000..bf620d04f11 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java @@ -0,0 +1,108 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.client; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.MapMaker; +import com.google.common.collect.Sets; +import com.metamx.common.ISE; +import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.emitter.EmittingLogger; +import org.apache.curator.framework.CuratorFramework; + +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; + +/** + */ +public class BatchingServerInventoryView extends ServerInventoryView> +{ + private static final EmittingLogger log = new EmittingLogger(BatchingServerInventoryView.class); + + final ConcurrentMap> zNodes = new MapMaker().makeMap(); + + public BatchingServerInventoryView( + final ServerInventoryViewConfig config, + final ZkPathsConfig zkPaths, + final CuratorFramework curator, + final ExecutorService exec, + final ObjectMapper jsonMapper + ) + { + super( + config, zkPaths, curator, exec, jsonMapper, new TypeReference>() + { + } + ); + } + + @Override + protected DruidServer addInnerInventory( + final DruidServer container, + String inventoryKey, + final Set inventory + ) + { + zNodes.put(inventoryKey, inventory); + for (DataSegment segment : inventory) { + addSingleInventory(container, segment); + } + return container; + } + + @Override + protected DruidServer updateInnerInventory( + DruidServer container, String inventoryKey, Set inventory + ) + { + Set existing = zNodes.get(inventoryKey); + if (existing == null) { + throw new ISE("Trying to update an inventoryKey[%s] that didn't exist?!", inventoryKey); + } + + for (DataSegment segment : Sets.difference(inventory, existing)) { + addSingleInventory(container, segment); + } + for (DataSegment segment : Sets.difference(existing, inventory)) { + removeSingleInventory(container, segment.getIdentifier()); + } + + return container; + } + + @Override + protected DruidServer removeInnerInventory(final DruidServer container, String inventoryKey) + { + log.info("Server[%s] removed container[%s]", container.getName(), inventoryKey); + Set segments = zNodes.remove(inventoryKey); + + if (segments == null) { + log.warn("Told to remove container[%s], which didn't exist", inventoryKey); + return container; + } + + for (DataSegment segment : segments) { + removeSingleInventory(container, segment.getIdentifier()); + } + return container; + } +} diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java index c98091e0560..b6639a2cb2a 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java @@ -20,6 +20,7 @@ package com.metamx.druid.client; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; @@ -43,27 +44,30 @@ import java.util.concurrent.atomic.AtomicBoolean; /** */ -public class ServerInventoryView implements ServerView, InventoryView +public abstract class ServerInventoryView implements ServerView, InventoryView { - private static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class); + protected static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class); - private final CuratorInventoryManager inventoryManager; + private ServerInventoryViewConfig config; + private final CuratorInventoryManager inventoryManager; private final AtomicBoolean started = new AtomicBoolean(false); - private final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); - private final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); + protected final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); + protected final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); - private static final Map removedSegments = new MapMaker().makeMap(); + protected static final Map removedSegments = new MapMaker().makeMap(); public ServerInventoryView( final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, final CuratorFramework curator, final ExecutorService exec, - final ObjectMapper jsonMapper + final ObjectMapper jsonMapper, + final TypeReference typeReference ) { - inventoryManager = new CuratorInventoryManager( + this.config = config; + this.inventoryManager = new CuratorInventoryManager( curator, new InventoryManagerConfig() { @@ -80,7 +84,7 @@ public class ServerInventoryView implements ServerView, InventoryView } }, exec, - new CuratorInventoryManagerStrategy() + new CuratorInventoryManagerStrategy() { @Override public DruidServer deserializeContainer(byte[] bytes) @@ -105,10 +109,10 @@ public class ServerInventoryView implements ServerView, InventoryView } @Override - public DataSegment deserializeInventory(byte[] bytes) + public InventoryType deserializeInventory(byte[] bytes) { try { - return jsonMapper.readValue(bytes, DataSegment.class); + return jsonMapper.readValue(bytes, typeReference); } catch (IOException e) { throw Throwables.propagate(e); @@ -116,7 +120,7 @@ public class ServerInventoryView implements ServerView, InventoryView } @Override - public byte[] serializeInventory(DataSegment inventory) + public byte[] serializeInventory(InventoryType inventory) { try { return jsonMapper.writeValueAsBytes(inventory); @@ -146,67 +150,27 @@ public class ServerInventoryView implements ServerView, InventoryView } @Override - public DruidServer addInventory(final DruidServer container, String inventoryKey, final DataSegment inventory) + public DruidServer addInventory( + final DruidServer container, + String inventoryKey, + final InventoryType inventory + ) { - log.info("Server[%s] added segment[%s]", container.getName(), inventoryKey); + return addInnerInventory(container, inventoryKey, inventory); + } - if (container.getSegment(inventoryKey) != null) { - log.warn( - "Not adding or running callbacks for existing segment[%s] on server[%s]", - inventoryKey, - container.getName() - ); - - return container; - } - - final DruidServer retVal = container.addDataSegment(inventoryKey, inventory); - - runSegmentCallbacks( - new Function() - { - @Override - public CallbackAction apply(SegmentCallback input) - { - return input.segmentAdded(container, inventory); - } - } - ); - - return retVal; + @Override + public DruidServer updateInventory( + DruidServer container, String inventoryKey, InventoryType inventory + ) + { + return updateInnerInventory(container, inventoryKey, inventory); } @Override public DruidServer removeInventory(final DruidServer container, String inventoryKey) { - log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey); - final DataSegment segment = container.getSegment(inventoryKey); - - if (segment == null) { - log.warn( - "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]", - inventoryKey, - container.getName() - ); - - return container; - } - - final DruidServer retVal = container.removeDataSegment(inventoryKey); - - runSegmentCallbacks( - new Function() - { - @Override - public CallbackAction apply(SegmentCallback input) - { - return input.segmentRemoved(container, segment); - } - } - ); - - removedSegments.put(inventoryKey, config.getRemovedSegmentLifetime()); - return retVal; + return removeInnerInventory(container, inventoryKey); } } ); @@ -282,7 +246,7 @@ public class ServerInventoryView implements ServerView, InventoryView segmentCallbacks.put(callback, exec); } - private void runSegmentCallbacks( + protected void runSegmentCallbacks( final Function fn ) { @@ -302,7 +266,7 @@ public class ServerInventoryView implements ServerView, InventoryView } } - private void runServerCallbacks(final DruidServer server) + protected void runServerCallbacks(final DruidServer server) { for (final Map.Entry entry : serverCallbacks.entrySet()) { entry.getValue().execute( @@ -319,4 +283,83 @@ public class ServerInventoryView implements ServerView, InventoryView ); } } + + protected void addSingleInventory( + final DruidServer container, + final DataSegment inventory + ) + { + log.info("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); + + if (container.getSegment(inventory.getIdentifier()) != null) { + log.warn( + "Not adding or running callbacks for existing segment[%s] on server[%s]", + inventory.getIdentifier(), + container.getName() + ); + + return; + } + + container.addDataSegment(inventory.getIdentifier(), inventory); + + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentAdded(container, inventory); + } + } + ); + } + + protected void removeSingleInventory(final DruidServer container, String inventoryKey) + { + log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey); + final DataSegment segment = container.getSegment(inventoryKey); + + if (segment == null) { + log.warn( + "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]", + inventoryKey, + container.getName() + ); + + return; + } + + container.removeDataSegment(inventoryKey); + + runSegmentCallbacks( + new Function() + { + @Override + public CallbackAction apply(SegmentCallback input) + { + return input.segmentRemoved(container, segment); + } + } + ); + + removedSegments.put(inventoryKey, config.getRemovedSegmentLifetime()); + } + + protected abstract DruidServer addInnerInventory( + final DruidServer container, + String inventoryKey, + final InventoryType inventory + ); + + protected abstract DruidServer updateInnerInventory( + final DruidServer container, + String inventoryKey, + final InventoryType inventory + ); + + protected abstract DruidServer removeInnerInventory( + final DruidServer container, + String inventoryKey + ); } diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java index 68de158cbf0..6130a96a66c 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java @@ -29,4 +29,8 @@ public abstract class ServerInventoryViewConfig @Config("druid.master.removedSegmentLifetime") @Default("1") public abstract int getRemovedSegmentLifetime(); + + @Config("druid.announcer.type") + @Default("legacy") + public abstract String getAnnouncerType(); } \ No newline at end of file diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java new file mode 100644 index 00000000000..30bdc30e46e --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java @@ -0,0 +1,71 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.client; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.initialization.ZkPathsConfig; +import org.apache.curator.framework.CuratorFramework; + +import java.util.concurrent.ExecutorService; + +/** + */ +public class SingleServerInventoryView extends ServerInventoryView +{ + public SingleServerInventoryView( + final ServerInventoryViewConfig config, + final ZkPathsConfig zkPaths, + final CuratorFramework curator, + final ExecutorService exec, + final ObjectMapper jsonMapper + ) + { + super( + config, zkPaths, curator, exec, jsonMapper, new TypeReference() + { + } + ); + } + + @Override + protected DruidServer addInnerInventory( + DruidServer container, String inventoryKey, DataSegment inventory + ) + { + addSingleInventory(container, inventory); + return container; + } + + @Override + protected DruidServer updateInnerInventory( + DruidServer container, String inventoryKey, DataSegment inventory + ) + { + return addInnerInventory(container, inventoryKey, inventory); + } + + @Override + protected DruidServer removeInnerInventory(DruidServer container, String inventoryKey) + { + removeSingleInventory(container, inventoryKey); + return container; + } +} diff --git a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java b/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java index ab1e31bbc49..2f713f87554 100644 --- a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java +++ b/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java @@ -290,11 +290,18 @@ public class CuratorInventoryManager switch (event.getType()) { case CHILD_ADDED: - case CHILD_UPDATED: - final InventoryClass inventory = strategy.deserializeInventory(child.getData()); + final InventoryClass addedInventory = strategy.deserializeInventory(child.getData()); synchronized (holder) { - holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, inventory)); + holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, addedInventory)); + } + + break; + case CHILD_UPDATED: + final InventoryClass updatedInventory = strategy.deserializeInventory(child.getData()); + + synchronized (holder) { + holder.setContainer(strategy.updateInventory(holder.getContainer(), inventoryKey, updatedInventory)); } break; diff --git a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java b/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java index 516045537ac..8cab619e16f 100644 --- a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java +++ b/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java @@ -33,5 +33,6 @@ public interface CuratorInventoryManagerStrategy public void deadContainer(ContainerClass deadContainer); public ContainerClass updateContainer(ContainerClass oldContainer, ContainerClass newContainer); public ContainerClass addInventory(ContainerClass container, String inventoryKey, InventoryClass inventory); + public ContainerClass updateInventory(ContainerClass container, String inventoryKey, InventoryClass inventory); public ContainerClass removeInventory(ContainerClass container, String inventoryKey); } diff --git a/client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java b/client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java index d4390587268..dc093e53cfb 100644 --- a/client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java +++ b/client/src/test/java/com/metamx/druid/coordination/BatchingCuratorDataSegmentAnnouncerTest.java @@ -111,6 +111,12 @@ public class BatchingCuratorDataSegmentAnnouncerTest { return 100000; } + + @Override + public String getAnnouncerType() + { + return "batch"; + } }, announcer, jsonMapper diff --git a/client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java b/client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java index 2d520418824..da72db06048 100644 --- a/client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java +++ b/client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java @@ -209,6 +209,14 @@ public class CuratorInventoryManagerTest extends CuratorTestBase return container; } + @Override + public Map updateInventory( + Map container, String inventoryKey, Integer inventory + ) + { + return addInventory(container, inventoryKey, inventory); + } + @Override public Map removeInventory(Map container, String inventoryKey) { diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 6c48aba9021..0ced574db34 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -27,13 +27,16 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.IAE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.BatchingServerInventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; +import com.metamx.druid.client.SingleServerInventoryView; import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.config.ConfigManager; @@ -131,9 +134,31 @@ public class MasterMain final ExecutorService exec = Executors.newFixedThreadPool( 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() ); - ServerInventoryView serverInventoryView = new ServerInventoryView( - configFactory.build(ServerInventoryViewConfig.class), zkPaths, curatorFramework, exec, jsonMapper - ); + + final ServerInventoryViewConfig serverInventoryViewConfig = configFactory.build(ServerInventoryViewConfig.class); + final String announcerType = serverInventoryViewConfig.getAnnouncerType(); + + final ServerInventoryView serverInventoryView; + if ("legacy".equalsIgnoreCase(announcerType)) { + serverInventoryView = new SingleServerInventoryView( + serverInventoryViewConfig, + zkPaths, + curatorFramework, + exec, + jsonMapper + ); + } else if ("batch".equalsIgnoreCase(announcerType)) { + serverInventoryView = new BatchingServerInventoryView( + serverInventoryViewConfig, + zkPaths, + curatorFramework, + exec, + jsonMapper + ); + } else { + throw new IAE("Unknown type %s", announcerType); + } + lifecycle.addManagedInstance(serverInventoryView); final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index e0f7b5841ae..e14c3b5a7b8 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -85,7 +85,7 @@ public class DruidMaster private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; private final DatabaseSegmentManager databaseSegmentManager; - private final ServerInventoryView serverInventoryView; + private final ServerInventoryView serverInventoryView; private final DatabaseRuleManager databaseRuleManager; private final CuratorFramework curator; private final ServiceEmitter emitter; diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index a497811e066..5a42d3e9b91 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.MapMaker; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.ServerInventoryView; +import com.metamx.druid.client.SingleServerInventoryView; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.metrics.NoopServiceEmitter; @@ -44,7 +44,7 @@ public class DruidMasterTest private CuratorFramework curator; private LoadQueueTaskMaster taskMaster; private DatabaseSegmentManager databaseSegmentManager; - private ServerInventoryView serverInventoryView; + private SingleServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; private DataSegment segment; @@ -58,7 +58,7 @@ public class DruidMasterTest segment = EasyMock.createNiceMock(DataSegment.class); loadQueuePeon = EasyMock.createNiceMock(LoadQueuePeon.class); loadManagementPeons = new MapMaker().makeMap(); - serverInventoryView = EasyMock.createMock(ServerInventoryView.class); + serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); databaseSegmentManager = EasyMock.createNiceMock(DatabaseSegmentManager.class); EasyMock.replay(databaseSegmentManager); From ef27810458c9b76d45f218cd5a7c480e285c40fe Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 22 Jul 2013 18:02:10 -0700 Subject: [PATCH 57/68] rename to fix case issue --- client/src/main/java/com/metamx/druid/QueryableNode.java | 4 ++-- ...rverInventoryVIew.java => BatchServerInventoryView.java} | 6 +++--- server/src/main/java/com/metamx/druid/http/MasterMain.java | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) rename client/src/main/java/com/metamx/druid/client/{BatchingServerInventoryVIew.java => BatchServerInventoryView.java} (95%) diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 1741e13a27e..1971bc5158e 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -34,7 +34,7 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.BatchingServerInventoryView; +import com.metamx.druid.client.BatchServerInventoryView; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerInventoryView; @@ -374,7 +374,7 @@ public abstract class QueryableNode extends Registering getJsonMapper() ); } else if ("batch".equalsIgnoreCase(announcerType)) { - serverInventoryView = new BatchingServerInventoryView( + serverInventoryView = new BatchServerInventoryView( serverInventoryViewConfig, getZkPaths(), getCuratorFramework(), diff --git a/client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java similarity index 95% rename from client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java rename to client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java index bf620d04f11..cc178862928 100644 --- a/client/src/main/java/com/metamx/druid/client/BatchingServerInventoryVIew.java +++ b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java @@ -34,13 +34,13 @@ import java.util.concurrent.ExecutorService; /** */ -public class BatchingServerInventoryView extends ServerInventoryView> +public class BatchServerInventoryView extends ServerInventoryView> { - private static final EmittingLogger log = new EmittingLogger(BatchingServerInventoryView.class); + private static final EmittingLogger log = new EmittingLogger(BatchServerInventoryView.class); final ConcurrentMap> zNodes = new MapMaker().makeMap(); - public BatchingServerInventoryView( + public BatchServerInventoryView( final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, final CuratorFramework curator, diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 0ced574db34..63521717e79 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -33,7 +33,7 @@ import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.BatchingServerInventoryView; +import com.metamx.druid.client.BatchServerInventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.SingleServerInventoryView; @@ -148,7 +148,7 @@ public class MasterMain jsonMapper ); } else if ("batch".equalsIgnoreCase(announcerType)) { - serverInventoryView = new BatchingServerInventoryView( + serverInventoryView = new BatchServerInventoryView( serverInventoryViewConfig, zkPaths, curatorFramework, From 980b09d9037c6909895e4d86b9bfcb7d1397a128 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Jul 2013 10:19:26 -0700 Subject: [PATCH 58/68] fix broken test --- .../com/metamx/druid/master/DruidMasterTest.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index 5a42d3e9b91..a49dc85a582 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -24,6 +24,7 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.SingleServerInventoryView; +import com.metamx.druid.curator.inventory.InventoryManagerConfig; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.metrics.NoopServiceEmitter; @@ -169,6 +170,20 @@ public class DruidMasterTest EasyMock.expect(serverInventoryView.getInventoryValue("from")).andReturn(druidServer); EasyMock.expect(serverInventoryView.getInventoryValue("to")).andReturn(druidServer); + EasyMock.expect(serverInventoryView.getInventoryManagerConfig()).andReturn(new InventoryManagerConfig() + { + @Override + public String getContainerPath() + { + return ""; + } + + @Override + public String getInventoryPath() + { + return ""; + } + }); EasyMock.replay(serverInventoryView); master.moveSegment("from", "to", "dummySegment", null); From 94afe7213347415cdac277fb163f93c95fd58768 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Jul 2013 12:49:07 -0700 Subject: [PATCH 59/68] add unit test --- .../client/BatchServerInventoryViewTest.java | 231 ++++++++++++++++++ 1 file changed, 231 insertions(+) create mode 100644 client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java diff --git a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java b/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java new file mode 100644 index 00000000000..7690b0ab06b --- /dev/null +++ b/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java @@ -0,0 +1,231 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Stopwatch; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.ISE; +import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; +import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; +import com.metamx.druid.curator.announcement.Announcer; +import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; +import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.druid.jackson.DefaultObjectMapper; +import junit.framework.Assert; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.test.TestingCluster; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + */ +public class BatchServerInventoryViewTest +{ + private static final String testBasePath = "/test"; + private static final Joiner joiner = Joiner.on("/"); + + private TestingCluster testingCluster; + private CuratorFramework cf; + private ObjectMapper jsonMapper; + private Announcer announcer; + private BatchDataSegmentAnnouncer segmentAnnouncer; + private Set testSegments; + private BatchServerInventoryView batchServerInventoryView; + + @Before + public void setUp() throws Exception + { + testingCluster = new TestingCluster(1); + testingCluster.start(); + + cf = CuratorFrameworkFactory.builder() + .connectString(testingCluster.getConnectString()) + .retryPolicy(new ExponentialBackoffRetry(1, 10)) + .compressionProvider(new PotentiallyGzippedCompressionProvider(true)) + .build(); + cf.start(); + cf.create().creatingParentsIfNeeded().forPath(testBasePath); + + jsonMapper = new DefaultObjectMapper(); + + announcer = new Announcer( + cf, + MoreExecutors.sameThreadExecutor() + ); + announcer.start(); + + segmentAnnouncer = new BatchDataSegmentAnnouncer( + new DruidServerMetadata( + "id", + "host", + Long.MAX_VALUE, + "type", + "tier" + ), + new ZkDataSegmentAnnouncerConfig() + { + @Override + public String getZkBasePath() + { + return testBasePath; + } + + @Override + public int getSegmentsPerNode() + { + return 50; + } + + @Override + public long getMaxNumBytes() + { + return 100000; + } + + @Override + public String getAnnouncerType() + { + return "batch"; + } + }, + announcer, + jsonMapper + ); + segmentAnnouncer.start(); + + testSegments = Sets.newHashSet(); + for (int i = 0; i < 100; i++) { + testSegments.add(makeSegment(i)); + } + + batchServerInventoryView = new BatchServerInventoryView( + new ServerInventoryViewConfig() + { + @Override + public int getRemovedSegmentLifetime() + { + return 0; + } + + @Override + public String getAnnouncerType() + { + return "batch"; + } + }, + new ZkPathsConfig() + { + @Override + public String getZkBasePath() + { + return testBasePath; + } + }, + cf, + Executors.newSingleThreadExecutor(), + jsonMapper + ); + + batchServerInventoryView.start(); + } + + @After + public void tearDown() throws Exception + { + batchServerInventoryView.stop(); + segmentAnnouncer.stop(); + announcer.stop(); + cf.close(); + testingCluster.stop(); + } + + @Test + public void testRun() throws Exception + { + segmentAnnouncer.announceSegments(testSegments); + + waitForSync(); + + DruidServer server = Iterables.get(batchServerInventoryView.getInventory(), 0); + Set segments = Sets.newHashSet(server.getSegments().values()); + + Assert.assertEquals(testSegments, segments); + + DataSegment segment1 = makeSegment(101); + DataSegment segment2 = makeSegment(102); + + segmentAnnouncer.announceSegment(segment1); + segmentAnnouncer.announceSegment(segment2); + testSegments.add(segment1); + testSegments.add(segment2); + + waitForSync(); + + Assert.assertEquals(testSegments, Sets.newHashSet(server.getSegments().values())); + + segmentAnnouncer.unannounceSegment(segment1); + segmentAnnouncer.unannounceSegment(segment2); + testSegments.remove(segment1); + testSegments.remove(segment2); + + waitForSync(); + + Assert.assertEquals(testSegments, Sets.newHashSet(server.getSegments().values())); + } + + private DataSegment makeSegment(int offset) + { + return DataSegment.builder() + .dataSource("foo") + .interval( + new Interval( + new DateTime("2013-01-01").plusDays(offset), + new DateTime("2013-01-02").plusDays(offset) + ) + ) + .version(new DateTime().toString()) + .build(); + } + + private void waitForSync() throws Exception + { + Stopwatch stopwatch = new Stopwatch().start(); + while (Iterables.get(batchServerInventoryView.getInventory(), 0).getSegments().size() != testSegments.size()) { + Thread.sleep(500); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 5000) { + throw new ISE("BatchServerInventoryView is not updating"); + } + } + } +} From fc65648809dd946a9749392c2b644cac5c2d8c1f Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Jul 2013 13:21:14 -0700 Subject: [PATCH 60/68] fix a bug introduced in postgres support that breaks inserts for mysql --- .../druid/realtime/DbSegmentPublisher.java | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java index 67177e8f45c..c60723ece50 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java @@ -60,13 +60,22 @@ public class DbSegmentPublisher implements SegmentPublisher @Override public Void withHandle(Handle handle) throws Exception { - handle.createStatement( - String.format( - "INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - config.getSegmentTable() - ) - ) + String statement; + if (!handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL")) { + statement = String.format( + "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + config.getSegmentTable() + ); + } else { + statement = String.format( + "INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + config.getSegmentTable() + ); + } + + handle.createStatement(statement) .bind("id", segment.getIdentifier()) .bind("dataSource", segment.getDataSource()) .bind("created_date", new DateTime().toString()) From e4c2a2c705d3ec4ca954eda46233c5111a94f570 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Jul 2013 16:21:45 -0700 Subject: [PATCH 61/68] fix according to comments in code review --- .../druid/client/BatchServerInventoryView.java | 1 + .../metamx/druid/client/ServerInventoryView.java | 14 ++++++++------ .../druid/client/SingleServerInventoryView.java | 4 ++++ .../druid/initialization/Initialization.java | 1 - 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java index 301c4ebb16e..58497e293db 100644 --- a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java @@ -51,6 +51,7 @@ public class BatchServerInventoryView extends ServerInventoryView implements ServerView, InventoryView { - protected static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class); - - private ServerInventoryViewConfig config; + private final ServerInventoryViewConfig config; + private final Logger log; private final CuratorInventoryManager inventoryManager; private final AtomicBoolean started = new AtomicBoolean(false); - protected final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); - protected final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); + private final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); + private final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); - protected static final Map removedSegments = new MapMaker().makeMap(); + private final Map removedSegments = new MapMaker().makeMap(); public ServerInventoryView( final ServerInventoryViewConfig config, + final Logger log, final InventoryManagerConfig inventoryManagerConfig, final CuratorFramework curator, final ExecutorService exec, @@ -66,6 +67,7 @@ public abstract class ServerInventoryView implements ServerView, ) { this.config = config; + this.log = log; this.inventoryManager = new CuratorInventoryManager( curator, inventoryManagerConfig, diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java index f858fe7a3dc..4b345dc5a29 100644 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.curator.inventory.InventoryManagerConfig; import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; import java.util.concurrent.ExecutorService; @@ -31,6 +32,8 @@ import java.util.concurrent.ExecutorService; */ public class SingleServerInventoryView extends ServerInventoryView { + private static final EmittingLogger log = new EmittingLogger(SingleServerInventoryView.class); + public SingleServerInventoryView( final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, @@ -41,6 +44,7 @@ public class SingleServerInventoryView extends ServerInventoryView { super( config, + log, new InventoryManagerConfig() { @Override diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index cd0f1147cda..de29eabd550 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -197,7 +197,6 @@ public class Initialization .connectString(curatorConfig.getZkHosts()) .sessionTimeoutMs(curatorConfig.getZkSessionTimeoutMs()) .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) - // Don't compress stuff written just yet, need to get code deployed first. .compressionProvider(new PotentiallyGzippedCompressionProvider(curatorConfig.enableCompression())) .build(); From fa5f46057c0d19e7dbdb842db00f51d8356dcd8e Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 23 Jul 2013 16:25:07 -0700 Subject: [PATCH 62/68] fix spacing issue --- client/src/main/java/com/metamx/druid/QueryableNode.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index f45b473fdfa..ca638f44727 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -346,8 +346,7 @@ public abstract class QueryableNode extends Registering } } - private void - initializeInventoryView() + private void initializeInventoryView() { if (inventoryView == null) { initializeServerInventoryView(); From 09576ccd09cd36e6d25ece8f8ca0db6c616365a6 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 24 Jul 2013 09:41:20 -0700 Subject: [PATCH 63/68] [maven-release-plugin] prepare release druid-0.5.19 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 207458b37c1..434a9a02933 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/common/pom.xml b/common/pom.xml index d9d6249b5f9..46da405c6ca 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/examples/pom.xml b/examples/pom.xml index 9669767f81e..1857191d64c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 5c50eac9b54..f12d7fb179a 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 803c3bbb571..e669a787d66 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 64d4a2aa129..985062af9dd 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/pom.xml b/pom.xml index 0ba44f14093..2cabc640e52 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.19-SNAPSHOT + 0.5.19 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index aac1b971aa5..199b160397c 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/server/pom.xml b/server/pom.xml index 3d611f96f03..a25fd2c1a7b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 diff --git a/services/pom.xml b/services/pom.xml index cb73a90eb89..4f18525b975 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.19-SNAPSHOT + 0.5.19 com.metamx druid - 0.5.19-SNAPSHOT + 0.5.19 From cfc635de53226d2180ccf97546a9f0a966ec2d67 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 24 Jul 2013 09:41:30 -0700 Subject: [PATCH 64/68] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 434a9a02933..40d6c40da8b 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 46da405c6ca..eb3c3c744a0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 1857191d64c..f9bdca945a5 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index f12d7fb179a..0b372032a62 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index e669a787d66..721cf428b5a 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 985062af9dd..ef35c9f1c3f 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/pom.xml b/pom.xml index 2cabc640e52..3dd102990b8 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.19 + 0.5.20-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 199b160397c..cb726ab2fc2 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index a25fd2c1a7b..f1fc3c47895 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 4f18525b975..3b4b95499a9 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.19 + 0.5.20-SNAPSHOT com.metamx druid - 0.5.19 + 0.5.20-SNAPSHOT From e3879f0c778eb62674de70b9623b6ca7d850dcfc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Wed, 24 Jul 2013 23:05:30 +0000 Subject: [PATCH 65/68] Adding a dependency on Apache commons httpclient in the Druid client pom.xml. It currently gets pulled in through some dependencies but it should be explicitly set. --- client/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/client/pom.xml b/client/pom.xml index 40d6c40da8b..d47be87f6ef 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -68,6 +68,10 @@ commons-codec commons-codec + + commons-httpclient + commons-httpclient + org.skife.config config-magic From b3530fb15dc189ece8352fcbcd9082a6abba2df5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Wed, 24 Jul 2013 23:37:21 +0000 Subject: [PATCH 66/68] Adding a configuration property to disable setting an ACL on the segment pushed to S3. --- .../com/metamx/druid/loading/S3DataSegmentPusher.java | 8 ++++++-- .../metamx/druid/loading/S3DataSegmentPusherConfig.java | 4 ++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 9dacbe8b546..d9ac69e5c5b 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -80,7 +80,9 @@ public class S3DataSegmentPusher implements DataSegmentPusher final String outputBucket = config.getBucket(); toPush.setBucketName(outputBucket); toPush.setKey(outputKey + "/index.zip"); - toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + if (!config.getDisableAcl()) { + toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + } log.info("Pushing %s.", toPush); s3Client.putObject(outputBucket, toPush); @@ -96,7 +98,9 @@ public class S3DataSegmentPusher implements DataSegmentPusher S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); descriptorObject.setKey(outputKey + "/descriptor.json"); - descriptorObject.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + if (!config.getDisableAcl()) { + descriptorObject.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + } log.info("Pushing %s", descriptorObject); s3Client.putObject(outputBucket, descriptorObject); diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index a2cada422fb..3fbbe2d311f 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -32,4 +32,8 @@ public abstract class S3DataSegmentPusherConfig @Config("druid.pusher.s3.baseKey") @Default("") public abstract String getBaseKey(); + + @Config("druid.pusher.s3.disableAcl") + @Default("false") + public abstract boolean getDisableAcl(); } From 2842a408945f4e29942ac426801fde4f1c435e89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stef=C3=A1n=20Freyr=20Stef=C3=A1nsson?= Date: Wed, 24 Jul 2013 23:41:47 +0000 Subject: [PATCH 67/68] Fixed test that was failing on JDK 1.7 --- .../src/test/java/druid/examples/web/WebJsonSupplierTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java b/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java index c1bc5c4dc34..5bb72e2af17 100644 --- a/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java +++ b/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java @@ -21,11 +21,11 @@ package druid.examples.web; import org.junit.Test; -import java.net.UnknownHostException; +import java.io.IOException; public class WebJsonSupplierTest { - @Test(expected = UnknownHostException.class) + @Test(expected = IOException.class) public void checkInvalidUrl() throws Exception { From a909a31bcc4bc07a511c42b58373a5e4475bd21f Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 24 Jul 2013 19:25:32 -0700 Subject: [PATCH 68/68] better select sinks --- .../plumber/RealtimePlumberSchool.java | 38 +++++++++++++++---- .../metamx/druid/realtime/plumber/Sink.java | 5 +++ 2 files changed, 36 insertions(+), 7 deletions(-) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java index a429fbef9d5..0bd01226828 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -37,6 +38,8 @@ import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerView; @@ -50,6 +53,7 @@ import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.partition.SingleElementPartitionChunk; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -186,6 +190,9 @@ public class RealtimePlumberSchool implements PlumberSchool private volatile ScheduledExecutorService scheduledExecutor = null; private final Map sinks = Maps.newConcurrentMap(); + private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( + String.CASE_INSENSITIVE_ORDER + ); @Override public void startJob() @@ -219,6 +226,7 @@ public class RealtimePlumberSchool implements PlumberSchool try { segmentAnnouncer.announceSegment(retVal.getSegment()); sinks.put(truncatedTime, retVal); + sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), new SingleElementPartitionChunk(retVal)); } catch (IOException e) { log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) @@ -247,17 +255,23 @@ public class RealtimePlumberSchool implements PlumberSchool } }; + List> querySinks = Lists.newArrayList(); + for (Interval interval : query.getIntervals()) { + querySinks.addAll(sinkTimeline.lookup(interval)); + } + return toolchest.mergeResults( factory.mergeRunners( EXEC, FunctionalIterable - .create(sinks.values()) + .create(querySinks) .transform( - new Function>() + new Function, QueryRunner>() { @Override - public QueryRunner apply(Sink input) + public QueryRunner apply(TimelineObjectHolder holder) { + final Sink theSink = holder.getObject().getChunk(0).getObject(); return new SpecificSegmentQueryRunner( new MetricsEmittingQueryRunner( emitter, @@ -265,7 +279,7 @@ public class RealtimePlumberSchool implements PlumberSchool factory.mergeRunners( EXEC, Iterables.transform( - input, + theSink, new Function>() { @Override @@ -279,9 +293,9 @@ public class RealtimePlumberSchool implements PlumberSchool ), new SpecificSegmentSpec( new SegmentDescriptor( - input.getInterval(), - input.getSegment().getVersion(), - input.getSegment().getShardSpec().getPartitionNum() + holder.getInterval(), + theSink.getSegment().getVersion(), + theSink.getSegment().getShardSpec().getPartitionNum() ) ) ); @@ -442,6 +456,11 @@ public class RealtimePlumberSchool implements PlumberSchool Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants); sinks.put(sinkInterval.getStartMillis(), currSink); + sinkTimeline.add( + currSink.getInterval(), + currSink.getVersion(), + new SingleElementPartitionChunk(currSink) + ); segmentAnnouncer.announceSegment(currSink.getSegment()); } @@ -490,6 +509,11 @@ public class RealtimePlumberSchool implements PlumberSchool FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval())); log.info("Removing sinkKey %d for segment %s", sinkKey, sink.getSegment().getIdentifier()); sinks.remove(sinkKey); + sinkTimeline.remove( + sink.getInterval(), + sink.getVersion(), + new SingleElementPartitionChunk(sink) + ); synchronized (handoffCondition) { handoffCondition.notifyAll(); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java index a5dd4ae38a1..a1823b6c09a 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java @@ -90,6 +90,11 @@ public class Sink implements Iterable makeNewCurrIndex(interval.getStartMillis(), schema); } + public String getVersion() + { + return version; + } + public Interval getInterval() { return interval;