+{
+ @Override
+ public ByteBufferInputRowParser withParseSpec(ParseSpec parseSpec);
+}
diff --git a/api/src/main/java/io/druid/data/input/Committer.java b/api/src/main/java/io/druid/data/input/Committer.java
new file mode 100644
index 00000000000..c4bab0d63b7
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/Committer.java
@@ -0,0 +1,32 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+/**
+ * Committer includes a Runnable and a Jackson-serialized metadata object containing the offset
+ */
+public interface Committer extends Runnable
+{
+ /**
+ * @return A json serialized represenation of commit metadata,
+ * which needs to be serialized and deserialized by Jackson.
+ * Commit metadata can be a complex type, but we recommend keeping it to List/Map/"Primitive JSON" types
+ * */
+ public Object getMetadata();
+}
diff --git a/api/src/main/java/io/druid/data/input/Firehose.java b/api/src/main/java/io/druid/data/input/Firehose.java
new file mode 100644
index 00000000000..eff91cb0964
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/Firehose.java
@@ -0,0 +1,78 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import java.io.Closeable;
+
+/**
+ * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
+ * abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
+ * one of these and register it with the Main.
+ *
+ * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
+ * Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
+ * gets passed around as an Iterator.
+ *
+ * The implementation of this interface only needs to be minimally thread-safe. The three methods ##hasMore(),
+ * ##nextRow() and ##commit() are all called from the same thread. ##commit(), however, returns a callback
+ * which will be called on another thread, so the operations inside of that callback must be thread-safe.
+ *
+ */
+public interface Firehose extends Closeable
+{
+ /**
+ * Returns whether there are more rows to process. This is used to indicate that another item is immediately
+ * available via ##nextRow(). Thus, if the stream is still available but there are no new messages on it, this call
+ * should block until a new message is available.
+ *
+ * If something happens such that the stream is no longer available, this should return false.
+ *
+ * @return true if and when there is another row available, false if the stream has dried up
+ */
+ public boolean hasMore();
+
+ /**
+ * The next row available. Should only be called if hasMore returns true.
+ *
+ * @return The next row
+ */
+ public InputRow nextRow() ;
+
+ /**
+ * Returns a runnable that will "commit" everything read up to the point at which commit() is called. This is
+ * often equivalent to everything that has been read since the last commit() call (or instantiation of the object),
+ * but doesn't necessarily have to be.
+ *
+ * This method is called when the main processing loop starts to persist its current batch of things to process.
+ * The returned runnable will be run when the current batch has been successfully persisted, there is usually
+ * some time lag between when this method is called and when the runnable is run. The Runnable is also run on
+ * a separate thread so its operation should be thread-safe.
+ *
+ * The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
+ * been committed on the writer side of this interface protocol.
+ *
+ * A simple implementation of this interface might do nothing when run() is called
+ * (in which case the same do-nothing instance can be returned every time), or
+ * a more complex implementation might clean up temporary resources that are no longer needed
+ * because of InputRows delivered by prior calls to ##nextRow().
+ *
+ */
+ public Runnable commit();
+}
diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java
new file mode 100644
index 00000000000..45e187eeeff
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java
@@ -0,0 +1,41 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.metamx.common.parsers.ParseException;
+import io.druid.data.input.impl.InputRowParser;
+
+import java.io.IOException;
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+public interface FirehoseFactory
+{
+ /**
+ * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to
+ * call hasMore() on the returned Firehose (which might subsequently block).
+ *
+ * If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return
+ * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
+ * invalid configuration is preferred over returning null.
+ */
+ public Firehose connect(T parser) throws IOException, ParseException;
+
+}
diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactoryV2.java b/api/src/main/java/io/druid/data/input/FirehoseFactoryV2.java
new file mode 100644
index 00000000000..32057333529
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/FirehoseFactoryV2.java
@@ -0,0 +1,44 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.metamx.common.parsers.ParseException;
+import io.druid.data.input.impl.InputRowParser;
+
+import java.io.IOException;
+/**
+ * Initialization method that connects up the FirehoseV2. If this method returns successfully it should be safe to
+ * call start() on the returned FirehoseV2 (which might subsequently block).
+ *
+ * In contrast to V1 version, FirehoseFactoryV2 is able to pass an additional json-serialized object to FirehoseV2,
+ * which contains last commit metadata
+ *
+ *
+ * If this method returns null, then any attempt to call start(), advance(), currRow(), makeCommitter() and close() on the return
+ * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
+ * invalid configuration is preferred over returning null.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+public interface FirehoseFactoryV2
+{
+ public FirehoseV2 connect(T parser, Object lastCommit) throws IOException, ParseException;
+
+}
diff --git a/api/src/main/java/io/druid/data/input/FirehoseV2.java b/api/src/main/java/io/druid/data/input/FirehoseV2.java
new file mode 100644
index 00000000000..e1cf6c57411
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/FirehoseV2.java
@@ -0,0 +1,88 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import java.io.Closeable;
+/**
+ * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
+ * abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
+ * one of these and register it with the Main.
+ *
+ * In contrast to Firehose v1 version, FirehoseV2 will always operate in a "peek, then advance" manner.
+ * And the intended usage patttern is
+ * 1. Call start()
+ * 2. Read currRow()
+ * 3. Call advance()
+ * 4. If index should be committed: commit()
+ * 5. GOTO 2
+ *
+ * Note that commit() is being called *after* advance.
+ *
+ * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
+ * Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
+ * gets passed around as an Iterator.
+ *
+ * The implementation of this interface only needs to be minimally thread-safe. The methods ##start(), ##advance(),
+ * ##currRow() and ##makeCommitter() are all called from the same thread. ##makeCommitter(), however, returns a callback
+ * which will be called on another thread, so the operations inside of that callback must be thread-safe.
+ *
+ */
+public interface FirehoseV2 extends Closeable
+{
+ /**
+ * For initial start
+ * */
+ void start() throws Exception;
+
+ /**
+ * Advance the firehose to the next offset. Implementations of this interface should make sure that
+ * if advance() is called and throws out an exception, the next call to currRow() should return a
+ * null value.
+ *
+ * @return true if and when there is another row available, false if the stream has dried up
+ */
+ public boolean advance();
+
+ /**
+ * @return The current row
+ */
+ public InputRow currRow() ;
+
+ /**
+ * Returns a Committer that will "commit" everything read up to the point at which makeCommitter() is called.
+ *
+ * This method is called when the main processing loop starts to persist its current batch of things to process.
+ * The returned committer will be run when the current batch has been successfully persisted
+ * and the metadata the committer carries can also be persisted along with segment data. There is usually
+ * some time lag between when this method is called and when the runnable is run. The Runnable is also run on
+ * a separate thread so its operation should be thread-safe.
+ *
+ * Note that "correct" usage of this interface will always call advance() before commit() if the current row
+ * is considered in the commit.
+ *
+ * The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
+ * been committed on the writer side of this interface protocol.
+ *
+ * A simple implementation of this interface might do nothing when run() is called,
+ * and save proper commit information in metadata
+ *
+ */
+ public Committer makeCommitter();
+}
diff --git a/api/src/main/java/io/druid/data/input/InputRow.java b/api/src/main/java/io/druid/data/input/InputRow.java
new file mode 100644
index 00000000000..a1fcc9d5409
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/InputRow.java
@@ -0,0 +1,40 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import java.util.List;
+
+/**
+ * An InputRow is the interface definition of an event being input into the data ingestion layer.
+ *
+ * An InputRow is a Row with a self-describing list of the dimensions available. This list is used to
+ * implement "schema-less" data ingestion that allows the system to add new dimensions as they appear.
+ *
+ */
+public interface
+ InputRow extends Row
+{
+ /**
+ * Returns the dimensions that exist in this row.
+ *
+ * @return the dimensions that exist in this row.
+ */
+ public List getDimensions();
+}
diff --git a/api/src/main/java/io/druid/data/input/MapBasedInputRow.java b/api/src/main/java/io/druid/data/input/MapBasedInputRow.java
new file mode 100644
index 00000000000..f966267dd71
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/MapBasedInputRow.java
@@ -0,0 +1,68 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import org.joda.time.DateTime;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ */
+public class MapBasedInputRow extends MapBasedRow implements InputRow
+{
+ private final List dimensions;
+
+ public MapBasedInputRow(
+ long timestamp,
+ List dimensions,
+ Map event
+ )
+ {
+ super(timestamp, event);
+ this.dimensions = dimensions;
+ }
+
+ public MapBasedInputRow(
+ DateTime timestamp,
+ List dimensions,
+ Map event
+ )
+ {
+ super(timestamp, event);
+ this.dimensions = dimensions;
+ }
+
+ @Override
+ public List getDimensions()
+ {
+ return dimensions;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "MapBasedInputRow{" +
+ "timestamp=" + new DateTime(getTimestampFromEpoch()) +
+ ", event=" + getEvent() +
+ ", dimensions=" + dimensions +
+ '}';
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/MapBasedRow.java b/api/src/main/java/io/druid/data/input/MapBasedRow.java
new file mode 100644
index 00000000000..8738428387c
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/MapBasedRow.java
@@ -0,0 +1,205 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import com.metamx.common.IAE;
+import com.metamx.common.logger.Logger;
+import com.metamx.common.parsers.ParseException;
+import org.joda.time.DateTime;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ */
+public class MapBasedRow implements Row
+{
+ private static final Logger log = new Logger(MapBasedRow.class);
+ private static final Function TO_STRING_INCLUDING_NULL = new Function() {
+ @Override
+ public String apply(final Object o)
+ {
+ return String.valueOf(o);
+ }
+ };
+
+ private final DateTime timestamp;
+ private final Map event;
+
+ private static final Pattern LONG_PAT = Pattern.compile("[-|+]?\\d+");
+
+ @JsonCreator
+ public MapBasedRow(
+ @JsonProperty("timestamp") DateTime timestamp,
+ @JsonProperty("event") Map event
+ )
+ {
+ this.timestamp = timestamp;
+ this.event = event;
+ }
+
+ public MapBasedRow(
+ long timestamp,
+ Map event
+ )
+ {
+ this(new DateTime(timestamp), event);
+ }
+
+ @Override
+ public long getTimestampFromEpoch()
+ {
+ return timestamp.getMillis();
+ }
+
+ @JsonProperty
+ public DateTime getTimestamp()
+ {
+ return timestamp;
+ }
+
+ @JsonProperty
+ public Map getEvent()
+ {
+ return event;
+ }
+
+ @Override
+ public List getDimension(String dimension)
+ {
+ final Object dimValue = event.get(dimension);
+
+ if (dimValue == null) {
+ return Collections.emptyList();
+ } else if (dimValue instanceof List) {
+ // guava's toString function fails on null objects, so please do not use it
+ return Lists.transform(
+ (List) dimValue,
+ TO_STRING_INCLUDING_NULL);
+ } else {
+ return Collections.singletonList(String.valueOf(dimValue));
+ }
+ }
+
+ @Override
+ public Object getRaw(String dimension)
+ {
+ return event.get(dimension);
+ }
+
+ @Override
+ public float getFloatMetric(String metric)
+ {
+ Object metricValue = event.get(metric);
+
+ if (metricValue == null) {
+ return 0.0f;
+ }
+
+ if (metricValue instanceof Number) {
+ return ((Number) metricValue).floatValue();
+ } else if (metricValue instanceof String) {
+ try {
+ return Float.valueOf(((String) metricValue).replace(",", ""));
+ }
+ catch (Exception e) {
+ throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", metric, metricValue);
+ }
+ } else {
+ throw new ParseException("Unknown type[%s]", metricValue.getClass());
+ }
+ }
+
+ @Override
+ public long getLongMetric(String metric)
+ {
+ Object metricValue = event.get(metric);
+
+ if (metricValue == null) {
+ return 0L;
+ }
+
+ if (metricValue instanceof Number) {
+ return ((Number) metricValue).longValue();
+ } else if (metricValue instanceof String) {
+ try {
+ String s = ((String) metricValue).replace(",", "");
+ return LONG_PAT.matcher(s).matches() ? Long.valueOf(s) : Double.valueOf(s).longValue();
+ }
+ catch (Exception e) {
+ throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", metric, metricValue);
+ }
+ } else {
+ throw new ParseException("Unknown type[%s]", metricValue.getClass());
+ }
+ }
+
+ @Override
+ public String toString()
+ {
+ return "MapBasedRow{" +
+ "timestamp=" + timestamp +
+ ", event=" + event +
+ '}';
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ MapBasedRow that = (MapBasedRow) o;
+
+ if (!event.equals(that.event)) {
+ return false;
+ }
+ if (!timestamp.equals(that.timestamp)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = timestamp.hashCode();
+ result = 31 * result + event.hashCode();
+ return result;
+ }
+
+ @Override
+ public int compareTo(Row o)
+ {
+ return timestamp.compareTo(o.getTimestamp());
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/Row.java b/api/src/main/java/io/druid/data/input/Row.java
new file mode 100644
index 00000000000..914d8146597
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/Row.java
@@ -0,0 +1,94 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.metamx.common.parsers.ParseException;
+import org.joda.time.DateTime;
+
+import java.util.List;
+
+/**
+ * A Row of data. This can be used for both input and output into various parts of the system. It assumes
+ * that the user already knows the schema of the row and can query for the parts that they care about.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "version", defaultImpl = MapBasedRow.class)
+@JsonSubTypes(value = {
+ @JsonSubTypes.Type(name = "v1", value = MapBasedRow.class)
+})
+public interface Row extends Comparable
+{
+ /**
+ * Returns the timestamp from the epoch in milliseconds. If the event happened _right now_, this would return the
+ * same thing as System.currentTimeMillis();
+ *
+ * @return the timestamp from the epoch in milliseconds.
+ */
+ public long getTimestampFromEpoch();
+
+ /**
+ * Returns the timestamp from the epoch as an org.joda.time.DateTime. If the event happened _right now_, this would return the
+ * same thing as new DateTime();
+ *
+ * @return the timestamp from the epoch as an org.joda.time.DateTime object.
+ */
+ public DateTime getTimestamp();
+
+ /**
+ * Returns the list of dimension values for the given column name.
+ *
+ *
+ * @param dimension the column name of the dimension requested
+ *
+ * @return the list of values for the provided column name
+ */
+ public List getDimension(String dimension);
+
+ /**
+ * Returns the raw dimension value for the given column name. This is different from #getDimension which
+ * all values to strings before returning them.
+ *
+ * @param dimension the column name of the dimension requested
+ *
+ * @return the value of the provided column name
+ */
+ public Object getRaw(String dimension);
+
+ /**
+ * Returns the float value of the given metric column.
+ *
+ *
+ * @param metric the column name of the metric requested
+ *
+ * @return the float value for the provided column name.
+ */
+ public float getFloatMetric(String metric);
+
+ /**
+ * Returns the long value of the given metric column.
+ *
+ *
+ * @param metric the column name of the metric requested
+ *
+ * @return the long value for the provided column name.
+ */
+ public long getLongMetric(String metric);
+}
diff --git a/api/src/main/java/io/druid/data/input/Rows.java b/api/src/main/java/io/druid/data/input/Rows.java
new file mode 100644
index 00000000000..7c37b3cb522
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/Rows.java
@@ -0,0 +1,71 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.collect.Maps;
+import com.metamx.common.ISE;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ */
+public class Rows
+{
+ public static InputRow toCaseInsensitiveInputRow(final Row row, final List dimensions)
+ {
+ if (row instanceof MapBasedRow) {
+ MapBasedRow mapBasedRow = (MapBasedRow) row;
+
+ TreeMap caseInsensitiveMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
+ caseInsensitiveMap.putAll(mapBasedRow.getEvent());
+ return new MapBasedInputRow(
+ mapBasedRow.getTimestamp(),
+ dimensions,
+ caseInsensitiveMap
+ );
+ }
+ throw new ISE("Can only convert MapBasedRow objects because we are ghetto like that.");
+ }
+
+ /**
+ * @param timeStamp rollup up timestamp to be used to create group key
+ * @param inputRow input row
+ * @return groupKey for the given input row
+ */
+ public static List toGroupKey(long timeStamp, InputRow inputRow)
+ {
+ final Map> dims = Maps.newTreeMap();
+ for (final String dim : inputRow.getDimensions()) {
+ final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
+ if (dimValues.size() > 0) {
+ dims.put(dim, dimValues);
+ }
+ }
+ return ImmutableList.of(
+ timeStamp,
+ dims
+ );
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java b/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java
new file mode 100644
index 00000000000..4f52d6ca340
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java
@@ -0,0 +1,100 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.metamx.common.parsers.CSVParser;
+import com.metamx.common.parsers.ParseException;
+import com.metamx.common.parsers.Parser;
+
+import java.util.List;
+
+/**
+ */
+public class CSVParseSpec extends ParseSpec
+{
+ private final String listDelimiter;
+ private final List columns;
+
+ @JsonCreator
+ public CSVParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
+ @JsonProperty("listDelimiter") String listDelimiter,
+ @JsonProperty("columns") List columns
+ )
+ {
+ super(timestampSpec, dimensionsSpec);
+
+ this.listDelimiter = listDelimiter;
+ Preconditions.checkNotNull(columns, "columns");
+ for (String column : columns) {
+ Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
+ }
+
+ this.columns = columns;
+
+ verify(dimensionsSpec.getDimensionNames());
+ }
+
+ @JsonProperty
+ public String getListDelimiter()
+ {
+ return listDelimiter;
+ }
+
+ @JsonProperty("columns")
+ public List getColumns()
+ {
+ return columns;
+ }
+
+ @Override
+ public void verify(List usedCols)
+ {
+ for (String columnName : usedCols) {
+ Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
+ }
+ }
+
+ @Override
+ public Parser makeParser()
+ {
+ return new CSVParser(Optional.fromNullable(listDelimiter), columns);
+ }
+
+ @Override
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new CSVParseSpec(spec, getDimensionsSpec(), listDelimiter, columns);
+ }
+
+ @Override
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new CSVParseSpec(getTimestampSpec(), spec, listDelimiter, columns);
+ }
+
+ public ParseSpec withColumns(List cols)
+ {
+ return new CSVParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java b/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java
new file mode 100644
index 00000000000..70e14c98038
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java
@@ -0,0 +1,124 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.metamx.common.parsers.DelimitedParser;
+import com.metamx.common.parsers.ParseException;
+import com.metamx.common.parsers.Parser;
+
+import java.util.List;
+
+/**
+ */
+public class DelimitedParseSpec extends ParseSpec
+{
+ private final String delimiter;
+ private final String listDelimiter;
+ private final List columns;
+
+ @JsonCreator
+ public DelimitedParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
+ @JsonProperty("delimiter") String delimiter,
+ @JsonProperty("listDelimiter") String listDelimiter,
+ @JsonProperty("columns") List columns
+ )
+ {
+ super(timestampSpec, dimensionsSpec);
+
+ this.delimiter = delimiter;
+ this.listDelimiter = listDelimiter;
+ Preconditions.checkNotNull(columns, "columns");
+ this.columns = columns;
+ for (String column : this.columns) {
+ Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
+ }
+
+ verify(dimensionsSpec.getDimensionNames());
+ }
+
+ @JsonProperty("delimiter")
+ public String getDelimiter()
+ {
+ return delimiter;
+ }
+
+ @JsonProperty("listDelimiter")
+ public String getListDelimiter()
+ {
+ return listDelimiter;
+ }
+
+ @JsonProperty("columns")
+ public List getColumns()
+ {
+ return columns;
+ }
+
+ @Override
+ public void verify(List usedCols)
+ {
+ for (String columnName : usedCols) {
+ Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
+ }
+ }
+
+ @Override
+ public Parser makeParser()
+ {
+ Parser retVal = new DelimitedParser(
+ Optional.fromNullable(delimiter),
+ Optional.fromNullable(listDelimiter)
+ );
+ retVal.setFieldNames(columns);
+ return retVal;
+ }
+
+ @Override
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new DelimitedParseSpec(spec, getDimensionsSpec(), delimiter, listDelimiter, columns);
+ }
+
+ @Override
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new DelimitedParseSpec(getTimestampSpec(), spec, delimiter, listDelimiter, columns);
+ }
+
+ public ParseSpec withDelimiter(String delim)
+ {
+ return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delim, listDelimiter, columns);
+ }
+
+ public ParseSpec withListDelimiter(String delim)
+ {
+ return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, delim, columns);
+ }
+
+ public ParseSpec withColumns(List cols)
+ {
+ return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, listDelimiter, cols);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java
new file mode 100644
index 00000000000..ac674b6282c
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java
@@ -0,0 +1,114 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringDimensionSchema.class)
+@JsonSubTypes(value = {
+ @JsonSubTypes.Type(name = DimensionSchema.STRING_TYPE_NAME, value = StringDimensionSchema.class),
+ @JsonSubTypes.Type(name = DimensionSchema.LONG_TYPE_NAME, value = LongDimensionSchema.class),
+ @JsonSubTypes.Type(name = DimensionSchema.FLOAT_TYPE_NAME, value = FloatDimensionSchema.class),
+ @JsonSubTypes.Type(name = DimensionSchema.SPATIAL_TYPE_NAME, value = NewSpatialDimensionSchema.class),
+})
+public abstract class DimensionSchema
+{
+ public static final String STRING_TYPE_NAME = "string";
+ public static final String LONG_TYPE_NAME = "long";
+ public static final String FLOAT_TYPE_NAME = "float";
+ public static final String SPATIAL_TYPE_NAME = "spatial";
+
+
+ // main druid and druid-api should really use the same ValueType enum.
+ // merge them when druid-api is merged back into the main repo
+ public enum ValueType
+ {
+ FLOAT,
+ LONG,
+ STRING,
+ COMPLEX;
+
+ @JsonValue
+ @Override
+ public String toString()
+ {
+ return this.name().toUpperCase();
+ }
+
+ @JsonCreator
+ public static ValueType fromString(String name)
+ {
+ return valueOf(name.toUpperCase());
+ }
+ }
+
+ private final String name;
+
+ protected DimensionSchema(String name)
+ {
+ this.name = Preconditions.checkNotNull(name, "Dimension name cannot be null.");
+ }
+
+ @JsonProperty
+ public String getName()
+ {
+ return name;
+ };
+
+ @JsonIgnore
+ public abstract String getTypeName();
+
+ @JsonIgnore
+ public abstract ValueType getValueType();
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ DimensionSchema that = (DimensionSchema) o;
+
+ return name.equals(that.name);
+
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return name.hashCode();
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java b/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java
new file mode 100644
index 00000000000..923ca2e481a
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java
@@ -0,0 +1,234 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.metamx.common.parsers.ParserUtils;
+
+import javax.annotation.Nullable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+
+public class DimensionsSpec
+{
+ private final List dimensions;
+ private final Set dimensionExclusions;
+ private final Map dimensionSchemaMap;
+
+ public static List getDefaultSchemas(List dimNames)
+ {
+ return Lists.transform(
+ dimNames,
+ new Function()
+ {
+ @Override
+ public DimensionSchema apply(String input)
+ {
+ return new StringDimensionSchema(input);
+ }
+ }
+ );
+ }
+
+ public static DimensionSchema convertSpatialSchema(SpatialDimensionSchema spatialSchema)
+ {
+ return new NewSpatialDimensionSchema(spatialSchema.getDimName(), spatialSchema.getDims());
+ }
+
+ @JsonCreator
+ public DimensionsSpec(
+ @JsonProperty("dimensions") List dimensions,
+ @JsonProperty("dimensionExclusions") List dimensionExclusions,
+ @Deprecated @JsonProperty("spatialDimensions") List spatialDimensions
+ )
+ {
+ this.dimensions = dimensions == null
+ ? Lists.newArrayList()
+ : Lists.newArrayList(dimensions);
+
+ this.dimensionExclusions = (dimensionExclusions == null)
+ ? Sets.newHashSet()
+ : Sets.newHashSet(dimensionExclusions);
+
+ List spatialDims = (spatialDimensions == null)
+ ? Lists.newArrayList()
+ : spatialDimensions;
+
+ verify(spatialDims);
+
+ // Map for easy dimension name-based schema lookup
+ this.dimensionSchemaMap = new HashMap<>();
+ for (DimensionSchema schema : this.dimensions) {
+ dimensionSchemaMap.put(schema.getName(), schema);
+ }
+
+ for(SpatialDimensionSchema spatialSchema : spatialDims) {
+ DimensionSchema newSchema = DimensionsSpec.convertSpatialSchema(spatialSchema);
+ this.dimensions.add(newSchema);
+ dimensionSchemaMap.put(newSchema.getName(), newSchema);
+ }
+ }
+
+
+ @JsonProperty
+ public List getDimensions()
+ {
+ return dimensions;
+ }
+
+ @JsonProperty
+ public Set getDimensionExclusions()
+ {
+ return dimensionExclusions;
+ }
+
+ @Deprecated @JsonIgnore
+ public List getSpatialDimensions()
+ {
+ Iterable filteredList = Iterables.filter(
+ dimensions, NewSpatialDimensionSchema.class
+ );
+
+ Iterable transformedList = Iterables.transform(
+ filteredList,
+ new Function()
+ {
+ @Nullable
+ @Override
+ public SpatialDimensionSchema apply(NewSpatialDimensionSchema input)
+ {
+ return new SpatialDimensionSchema(input.getName(), input.getDims());
+ }
+ }
+ );
+
+ return Lists.newArrayList(transformedList);
+ }
+
+
+ @JsonIgnore
+ public List getDimensionNames()
+ {
+ return Lists.transform(
+ dimensions,
+ new Function()
+ {
+ @Override
+ public String apply(DimensionSchema input)
+ {
+ return input.getName();
+ }
+ }
+ );
+ }
+
+ public DimensionSchema getSchema(String dimension)
+ {
+ return dimensionSchemaMap.get(dimension);
+ }
+
+ public boolean hasCustomDimensions()
+ {
+ return !(dimensions == null || dimensions.isEmpty());
+ }
+
+ public DimensionsSpec withDimensions(List dims)
+ {
+ return new DimensionsSpec(dims, ImmutableList.copyOf(dimensionExclusions), null);
+ }
+
+ public DimensionsSpec withDimensionExclusions(Set dimExs)
+ {
+ return new DimensionsSpec(
+ dimensions,
+ ImmutableList.copyOf(Sets.union(dimensionExclusions, dimExs)),
+ null
+ );
+ }
+
+ @Deprecated
+ public DimensionsSpec withSpatialDimensions(List spatials)
+ {
+ return new DimensionsSpec(dimensions, ImmutableList.copyOf(dimensionExclusions), spatials);
+ }
+
+ private void verify(List spatialDimensions)
+ {
+ List dimNames = getDimensionNames();
+ Preconditions.checkArgument(
+ Sets.intersection(this.dimensionExclusions, Sets.newHashSet(dimNames)).isEmpty(),
+ "dimensions and dimensions exclusions cannot overlap"
+ );
+
+ ParserUtils.validateFields(dimNames);
+ ParserUtils.validateFields(dimensionExclusions);
+
+ List spatialDimNames = Lists.transform(
+ spatialDimensions,
+ new Function()
+ {
+ @Override
+ public String apply(SpatialDimensionSchema input)
+ {
+ return input.getDimName();
+ }
+ }
+ );
+
+ // Don't allow duplicates between main list and deprecated spatial list
+ ParserUtils.validateFields(Iterables.concat(dimNames, spatialDimNames));
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ DimensionsSpec that = (DimensionsSpec) o;
+
+ if (!dimensions.equals(that.dimensions)) {
+ return false;
+ }
+
+ return dimensionExclusions.equals(that.dimensionExclusions);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = dimensions.hashCode();
+ result = 31 * result + dimensionExclusions.hashCode();
+ return result;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java
new file mode 100644
index 00000000000..e4c562d1364
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to Metamarkets Group Inc. (Metamarkets) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. Metamarkets licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.google.common.base.Throwables;
+import io.druid.data.input.Firehose;
+import io.druid.data.input.InputRow;
+import io.druid.utils.Runnables;
+import org.apache.commons.io.LineIterator;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ */
+public class FileIteratingFirehose implements Firehose
+{
+ private final Iterator lineIterators;
+ private final StringInputRowParser parser;
+
+ private LineIterator lineIterator = null;
+
+ public FileIteratingFirehose(
+ Iterator lineIterators,
+ StringInputRowParser parser
+ )
+ {
+ this.lineIterators = lineIterators;
+ this.parser = parser;
+ }
+
+ @Override
+ public boolean hasMore()
+ {
+ while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) {
+ lineIterator = lineIterators.next();
+ }
+
+ return lineIterator != null && lineIterator.hasNext();
+ }
+
+ @Override
+ public InputRow nextRow()
+ {
+ try {
+ if (lineIterator == null || !lineIterator.hasNext()) {
+ // Close old streams, maybe.
+ if (lineIterator != null) {
+ lineIterator.close();
+ }
+
+ lineIterator = lineIterators.next();
+ }
+
+ return parser.parse(lineIterator.next());
+ }
+ catch (Exception e) {
+ throw Throwables.propagate(e);
+ }
+ }
+
+ @Override
+ public Runnable commit()
+ {
+ return Runnables.getNoopRunnable();
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ if (lineIterator != null) {
+ lineIterator.close();
+ }
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java
new file mode 100644
index 00000000000..db3b04a631b
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java
@@ -0,0 +1,48 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class FloatDimensionSchema extends DimensionSchema
+{
+ @JsonCreator
+ public FloatDimensionSchema(
+ @JsonProperty("name") String name
+ )
+ {
+ super(name);
+ }
+
+ @Override
+ public String getTypeName()
+ {
+ return DimensionSchema.FLOAT_TYPE_NAME;
+ }
+
+ @Override
+ @JsonIgnore
+ public ValueType getValueType()
+ {
+ return ValueType.FLOAT;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/InputRowParser.java b/api/src/main/java/io/druid/data/input/impl/InputRowParser.java
new file mode 100644
index 00000000000..15c77c6d1ab
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/InputRowParser.java
@@ -0,0 +1,38 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.metamx.common.parsers.ParseException;
+import io.druid.data.input.InputRow;
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class)
+@JsonSubTypes(value = {
+ @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class),
+ @JsonSubTypes.Type(name = "map", value = MapInputRowParser.class),
+ @JsonSubTypes.Type(name = "noop", value = NoopInputRowParser.class)
+})
+public interface InputRowParser
+{
+ public InputRow parse(T input) ;
+
+ public ParseSpec getParseSpec();
+
+ public InputRowParser withParseSpec(ParseSpec parseSpec) ;
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/JSONLowercaseParseSpec.java b/api/src/main/java/io/druid/data/input/impl/JSONLowercaseParseSpec.java
new file mode 100644
index 00000000000..e90b8262467
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/JSONLowercaseParseSpec.java
@@ -0,0 +1,68 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.metamx.common.parsers.JSONToLowerParser;
+import com.metamx.common.parsers.Parser;
+
+import java.util.List;
+
+/**
+ * This class is only here for backwards compatibility
+ */
+@Deprecated
+public class JSONLowercaseParseSpec extends ParseSpec
+{
+ private final ObjectMapper objectMapper;
+
+ @JsonCreator
+ public JSONLowercaseParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec
+ )
+ {
+ super(timestampSpec, dimensionsSpec);
+ this.objectMapper = new ObjectMapper();
+ }
+
+ @Override
+ public void verify(List usedCols)
+ {
+ }
+
+ @Override
+ public Parser makeParser()
+ {
+ return new JSONToLowerParser(objectMapper, null, null);
+ }
+
+ @Override
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new JSONLowercaseParseSpec(spec, getDimensionsSpec());
+ }
+
+ @Override
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new JSONLowercaseParseSpec(getTimestampSpec(), spec);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java b/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java
new file mode 100644
index 00000000000..907a254f082
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java
@@ -0,0 +1,128 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonParser.Feature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.metamx.common.parsers.JSONPathParser;
+import com.metamx.common.parsers.Parser;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ */
+public class JSONParseSpec extends ParseSpec
+{
+ private final ObjectMapper objectMapper;
+ private final JSONPathSpec flattenSpec;
+ private final Map featureSpec;
+
+ @JsonCreator
+ public JSONParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
+ @JsonProperty("flattenSpec") JSONPathSpec flattenSpec,
+ @JsonProperty("featureSpec") Map featureSpec
+ )
+ {
+ super(timestampSpec, dimensionsSpec);
+ this.objectMapper = new ObjectMapper();
+ this.flattenSpec = flattenSpec != null ? flattenSpec : new JSONPathSpec(true, null);
+ this.featureSpec = (featureSpec == null) ? new HashMap() : featureSpec;
+ for (Map.Entry entry : this.featureSpec.entrySet()) {
+ Feature feature = Feature.valueOf(entry.getKey());
+ objectMapper.configure(feature, entry.getValue());
+ }
+ }
+
+ @Deprecated
+ public JSONParseSpec(TimestampSpec ts, DimensionsSpec dims)
+ {
+ this(ts, dims, null, null);
+ }
+
+ @Override
+ public void verify(List usedCols)
+ {
+ }
+
+ @Override
+ public Parser makeParser()
+ {
+ return new JSONPathParser(
+ convertFieldSpecs(flattenSpec.getFields()),
+ flattenSpec.isUseFieldDiscovery(),
+ objectMapper
+ );
+ }
+
+ @Override
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new JSONParseSpec(spec, getDimensionsSpec(), getFlattenSpec(), getFeatureSpec());
+ }
+
+ @Override
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new JSONParseSpec(getTimestampSpec(), spec, getFlattenSpec(), getFeatureSpec());
+ }
+
+ @JsonProperty
+ public JSONPathSpec getFlattenSpec()
+ {
+ return flattenSpec;
+ }
+
+ @JsonProperty
+ public Map getFeatureSpec()
+ {
+ return featureSpec;
+ }
+
+ private List convertFieldSpecs(List druidFieldSpecs)
+ {
+ List newSpecs = new ArrayList<>();
+ for (JSONPathFieldSpec druidSpec : druidFieldSpecs) {
+ JSONPathParser.FieldType type;
+ switch (druidSpec.getType()) {
+ case ROOT:
+ type = JSONPathParser.FieldType.ROOT;
+ break;
+ case PATH:
+ type = JSONPathParser.FieldType.PATH;
+ break;
+ default:
+ throw new IllegalArgumentException("Invalid type for field " + druidSpec.getName());
+ }
+
+ JSONPathParser.FieldSpec newSpec = new JSONPathParser.FieldSpec(
+ type,
+ druidSpec.getName(),
+ druidSpec.getExpr()
+ );
+ newSpecs.add(newSpec);
+ }
+ return newSpecs;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/JSONPathFieldSpec.java b/api/src/main/java/io/druid/data/input/impl/JSONPathFieldSpec.java
new file mode 100644
index 00000000000..a5d56d25e36
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/JSONPathFieldSpec.java
@@ -0,0 +1,76 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class JSONPathFieldSpec
+{
+ private final JSONPathFieldType type;
+ private final String name;
+ private final String expr;
+
+ @JsonCreator
+ public JSONPathFieldSpec(
+ @JsonProperty("type") JSONPathFieldType type,
+ @JsonProperty("name") String name,
+ @JsonProperty("expr") String expr
+ )
+ {
+ this.type = type;
+ this.name = name;
+ this.expr = expr;
+ }
+
+ @JsonProperty
+ public JSONPathFieldType getType()
+ {
+ return type;
+ }
+
+ @JsonProperty
+ public String getName()
+ {
+ return name;
+ }
+
+ @JsonProperty
+ public String getExpr()
+ {
+ return expr;
+ }
+
+ @JsonCreator
+ public static JSONPathFieldSpec fromString(String name)
+ {
+ return JSONPathFieldSpec.createRootField(name);
+ }
+
+ public static JSONPathFieldSpec createNestedField(String name, String expr)
+ {
+ return new JSONPathFieldSpec(JSONPathFieldType.PATH, name, expr);
+ }
+
+ public static JSONPathFieldSpec createRootField(String name)
+ {
+ return new JSONPathFieldSpec(JSONPathFieldType.ROOT, name, name);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/JSONPathFieldType.java b/api/src/main/java/io/druid/data/input/impl/JSONPathFieldType.java
new file mode 100644
index 00000000000..4f549c61944
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/JSONPathFieldType.java
@@ -0,0 +1,42 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+public enum JSONPathFieldType
+{
+ ROOT,
+ PATH;
+
+ @JsonValue
+ @Override
+ public String toString()
+ {
+ return this.name().toLowerCase();
+ }
+
+ @JsonCreator
+ public static JSONPathFieldType fromString(String name)
+ {
+ return valueOf(name.toUpperCase());
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/JSONPathSpec.java b/api/src/main/java/io/druid/data/input/impl/JSONPathSpec.java
new file mode 100644
index 00000000000..33c25b84195
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/JSONPathSpec.java
@@ -0,0 +1,54 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+public class JSONPathSpec
+{
+ private final boolean useFieldDiscovery;
+ private final List fields;
+
+ @JsonCreator
+ public JSONPathSpec(
+ @JsonProperty("useFieldDiscovery") Boolean useFieldDiscovery,
+ @JsonProperty("fields") List fields
+ )
+ {
+ this.useFieldDiscovery = useFieldDiscovery == null ? true : useFieldDiscovery;
+ this.fields = fields == null ? ImmutableList.of() : fields;
+ }
+
+ @JsonProperty
+ public boolean isUseFieldDiscovery()
+ {
+ return useFieldDiscovery;
+ }
+
+ @JsonProperty
+ public List getFields()
+ {
+ return fields;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/JavaScriptParseSpec.java b/api/src/main/java/io/druid/data/input/impl/JavaScriptParseSpec.java
new file mode 100644
index 00000000000..69b26555e16
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/JavaScriptParseSpec.java
@@ -0,0 +1,80 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.metamx.common.parsers.JavaScriptParser;
+import com.metamx.common.parsers.Parser;
+
+import java.util.List;
+
+/**
+ */
+public class JavaScriptParseSpec extends ParseSpec
+{
+ private final String function;
+
+ @JsonCreator
+ public JavaScriptParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
+ @JsonProperty("function") String function
+ )
+ {
+ super(timestampSpec, dimensionsSpec);
+
+ this.function = function;
+ }
+
+ @JsonProperty("function")
+ public String getFunction()
+ {
+ return function;
+ }
+
+ @Override
+ public void verify(List usedCols)
+ {
+ }
+
+ @Override
+ public Parser makeParser()
+ {
+ return new JavaScriptParser(function);
+ }
+
+ @Override
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new JavaScriptParseSpec(spec, getDimensionsSpec(), function);
+ }
+
+ @Override
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new JavaScriptParseSpec(getTimestampSpec(), spec, function);
+ }
+
+ public ParseSpec withFunction(String fn)
+ {
+ return new JavaScriptParseSpec(getTimestampSpec(), getDimensionsSpec(), fn);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java
new file mode 100644
index 00000000000..4fd77d46924
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java
@@ -0,0 +1,48 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class LongDimensionSchema extends DimensionSchema
+{
+ @JsonCreator
+ public LongDimensionSchema(
+ @JsonProperty("name") String name
+ )
+ {
+ super(name);
+ }
+
+ @Override
+ public String getTypeName()
+ {
+ return DimensionSchema.LONG_TYPE_NAME;
+ }
+
+ @Override
+ @JsonIgnore
+ public ValueType getValueType()
+ {
+ return ValueType.LONG;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java
new file mode 100644
index 00000000000..e94e27fd1eb
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java
@@ -0,0 +1,90 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.metamx.common.logger.Logger;
+import com.metamx.common.parsers.ParseException;
+import io.druid.data.input.InputRow;
+import io.druid.data.input.MapBasedInputRow;
+import org.joda.time.DateTime;
+
+import java.util.List;
+import java.util.Map;
+
+public class MapInputRowParser implements InputRowParser>
+{
+ private final ParseSpec parseSpec;
+
+ @JsonCreator
+ public MapInputRowParser(
+ @JsonProperty("parseSpec") ParseSpec parseSpec
+ )
+ {
+ this.parseSpec = parseSpec;
+ }
+
+ @Override
+ public InputRow parse(Map theMap)
+ {
+ final List dimensions = parseSpec.getDimensionsSpec().hasCustomDimensions()
+ ? parseSpec.getDimensionsSpec().getDimensionNames()
+ : Lists.newArrayList(
+ Sets.difference(
+ theMap.keySet(),
+ parseSpec.getDimensionsSpec()
+ .getDimensionExclusions()
+ )
+ );
+
+ final DateTime timestamp;
+ try {
+ timestamp = parseSpec.getTimestampSpec().extractTimestamp(theMap);
+ if (timestamp == null) {
+ final String input = theMap.toString();
+ throw new NullPointerException(
+ String.format(
+ "Null timestamp in input: %s",
+ input.length() < 100 ? input : input.substring(0, 100) + "..."
+ )
+ );
+ }
+ }
+ catch (Exception e) {
+ throw new ParseException(e, "Unparseable timestamp found!");
+ }
+
+ return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap);
+ }
+
+ @JsonProperty
+ @Override
+ public ParseSpec getParseSpec()
+ {
+ return parseSpec;
+ }
+
+ @Override
+ public InputRowParser withParseSpec(ParseSpec parseSpec)
+ {
+ return new MapInputRowParser(parseSpec);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java
new file mode 100644
index 00000000000..ae834262bb9
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java
@@ -0,0 +1,88 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ * NOTE:
+ * This class should be deprecated after Druid supports configurable index types on dimensions.
+ * When that exists, this should be the implementation: https://github.com/druid-io/druid/issues/2622
+ *
+ * This is a stop-gap solution to consolidate the dimension specs and remove the separate spatial
+ * section in DimensionsSpec.
+ */
+public class NewSpatialDimensionSchema extends DimensionSchema
+{
+ private final List dims;
+
+ @JsonCreator
+ public NewSpatialDimensionSchema(
+ @JsonProperty("name") String name,
+ @JsonProperty("dims") List dims
+ )
+ {
+ super(name);
+ this.dims = dims;
+ }
+
+ @JsonProperty
+ public List getDims()
+ {
+ return dims;
+ }
+
+ @Override
+ public String getTypeName()
+ {
+ return DimensionSchema.SPATIAL_TYPE_NAME;
+ }
+
+ @Override
+ @JsonIgnore
+ public ValueType getValueType()
+ {
+ return ValueType.STRING;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ NewSpatialDimensionSchema that = (NewSpatialDimensionSchema) o;
+
+ return dims != null ? dims.equals(that.dims) : that.dims == null;
+
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return dims != null ? dims.hashCode() : 0;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java
new file mode 100644
index 00000000000..e7937453299
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java
@@ -0,0 +1,79 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.druid.data.input.InputRow;
+
+/**
+ */
+public class NoopInputRowParser implements InputRowParser
+{
+ private final ParseSpec parseSpec;
+
+ @JsonCreator
+ public NoopInputRowParser(
+ @JsonProperty("parseSpec") ParseSpec parseSpec
+ )
+ {
+ this.parseSpec = parseSpec != null ? parseSpec : new TimeAndDimsParseSpec(null, null);
+ }
+
+ @Override
+ public InputRow parse(InputRow input)
+ {
+ return input;
+ }
+
+ @Override
+ public ParseSpec getParseSpec()
+ {
+ return parseSpec;
+ }
+
+ @Override
+ public InputRowParser withParseSpec(ParseSpec parseSpec)
+ {
+ return new NoopInputRowParser(parseSpec);
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ NoopInputRowParser that = (NoopInputRowParser) o;
+
+ return parseSpec.equals(that.parseSpec);
+
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return parseSpec.hashCode();
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/ParseSpec.java b/api/src/main/java/io/druid/data/input/impl/ParseSpec.java
new file mode 100644
index 00000000000..66a09d03f24
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/ParseSpec.java
@@ -0,0 +1,111 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.metamx.common.parsers.Parser;
+
+import java.util.List;
+
+/**
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format", defaultImpl = DelimitedParseSpec.class)
+@JsonSubTypes(value = {
+ @JsonSubTypes.Type(name = "json", value = JSONParseSpec.class),
+ @JsonSubTypes.Type(name = "csv", value = CSVParseSpec.class),
+ @JsonSubTypes.Type(name = "tsv", value = DelimitedParseSpec.class),
+ @JsonSubTypes.Type(name = "jsonLowercase", value = JSONLowercaseParseSpec.class),
+ @JsonSubTypes.Type(name = "timeAndDims", value = TimeAndDimsParseSpec.class),
+ @JsonSubTypes.Type(name = "regex", value = RegexParseSpec.class),
+ @JsonSubTypes.Type(name = "javascript", value = JavaScriptParseSpec.class)
+
+})
+public abstract class ParseSpec
+{
+ private final TimestampSpec timestampSpec;
+ private final DimensionsSpec dimensionsSpec;
+
+ protected ParseSpec(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec)
+ {
+ this.timestampSpec = timestampSpec;
+ this.dimensionsSpec = dimensionsSpec;
+ }
+
+ @JsonProperty
+ public TimestampSpec getTimestampSpec()
+ {
+ return timestampSpec;
+ }
+
+ @JsonProperty
+ public DimensionsSpec getDimensionsSpec()
+ {
+ return dimensionsSpec;
+ }
+
+ public void verify(List usedCols)
+ {
+ // do nothing
+ }
+
+ public Parser makeParser()
+ {
+ return null;
+ }
+
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ ParseSpec parseSpec = (ParseSpec) o;
+
+ if (timestampSpec != null ? !timestampSpec.equals(parseSpec.timestampSpec) : parseSpec.timestampSpec != null) {
+ return false;
+ }
+ return !(dimensionsSpec != null
+ ? !dimensionsSpec.equals(parseSpec.dimensionsSpec)
+ : parseSpec.dimensionsSpec != null);
+
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = timestampSpec != null ? timestampSpec.hashCode() : 0;
+ result = 31 * result + (dimensionsSpec != null ? dimensionsSpec.hashCode() : 0);
+ return result;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/RegexParseSpec.java b/api/src/main/java/io/druid/data/input/impl/RegexParseSpec.java
new file mode 100644
index 00000000000..992f8070da1
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/RegexParseSpec.java
@@ -0,0 +1,115 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.metamx.common.parsers.Parser;
+import com.metamx.common.parsers.RegexParser;
+
+import java.util.List;
+
+/**
+ */
+public class RegexParseSpec extends ParseSpec
+{
+ private final String listDelimiter;
+ private final List columns;
+ private final String pattern;
+
+ @JsonCreator
+ public RegexParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
+ @JsonProperty("listDelimiter") String listDelimiter,
+ @JsonProperty("columns") List columns,
+ @JsonProperty("pattern") String pattern
+ )
+ {
+ super(timestampSpec, dimensionsSpec);
+
+ this.listDelimiter = listDelimiter;
+ this.columns = columns;
+ this.pattern = pattern;
+
+ verify(dimensionsSpec.getDimensionNames());
+ }
+
+ @JsonProperty
+ public String getListDelimiter()
+ {
+ return listDelimiter;
+ }
+
+ @JsonProperty("pattern")
+ public String getPattern()
+ {
+ return pattern;
+ }
+
+ @JsonProperty
+ public List getColumns()
+ {
+ return columns;
+ }
+
+ @Override
+ public void verify(List usedCols)
+ {
+ if (columns != null) {
+ for (String columnName : usedCols) {
+ Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
+ }
+ }
+ }
+
+ @Override
+ public Parser makeParser()
+ {
+ if (columns == null) {
+ return new RegexParser(pattern, Optional.fromNullable(listDelimiter));
+ }
+ return new RegexParser(pattern, Optional.fromNullable(listDelimiter), columns);
+ }
+
+ @Override
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new RegexParseSpec(spec, getDimensionsSpec(), listDelimiter, columns, pattern);
+ }
+
+ @Override
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new RegexParseSpec(getTimestampSpec(), spec, listDelimiter, columns, pattern);
+ }
+
+ public ParseSpec withColumns(List cols)
+ {
+ return new RegexParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols, pattern);
+ }
+
+ public ParseSpec withPattern(String pat)
+ {
+ return new RegexParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, columns, pat);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/SpatialDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/SpatialDimensionSchema.java
new file mode 100644
index 00000000000..b815c1c6ec6
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/SpatialDimensionSchema.java
@@ -0,0 +1,82 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.List;
+
+/**
+ */
+@Deprecated
+public class SpatialDimensionSchema
+{
+ private final String dimName;
+ private final List dims;
+
+ @JsonCreator
+ public SpatialDimensionSchema(
+ @JsonProperty("dimName") String dimName,
+ @JsonProperty("dims") List dims
+ )
+ {
+ this.dimName = dimName;
+ this.dims = dims;
+ }
+
+ @JsonProperty
+ public String getDimName()
+ {
+ return dimName;
+ }
+
+ @JsonProperty
+ public List getDims()
+ {
+ return dims;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ SpatialDimensionSchema that = (SpatialDimensionSchema) o;
+
+ if (dimName != null ? !dimName.equals(that.dimName) : that.dimName != null) {
+ return false;
+ }
+ return dims != null ? dims.equals(that.dims) : that.dims == null;
+
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = dimName != null ? dimName.hashCode() : 0;
+ result = 31 * result + (dims != null ? dims.hashCode() : 0);
+ return result;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java
new file mode 100644
index 00000000000..02fef40dda4
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java
@@ -0,0 +1,53 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class StringDimensionSchema extends DimensionSchema
+{
+ @JsonCreator
+ public static StringDimensionSchema create(String name) {
+ return new StringDimensionSchema(name);
+ }
+
+ @JsonCreator
+ public StringDimensionSchema(
+ @JsonProperty("name") String name
+ )
+ {
+ super(name);
+ }
+
+ @Override
+ public String getTypeName()
+ {
+ return DimensionSchema.STRING_TYPE_NAME;
+ }
+
+ @Override
+ @JsonIgnore
+ public ValueType getValueType()
+ {
+ return ValueType.STRING;
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java
new file mode 100644
index 00000000000..3b2d048ff06
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/StringInputRowParser.java
@@ -0,0 +1,138 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Charsets;
+import com.metamx.common.parsers.ParseException;
+import com.metamx.common.parsers.Parser;
+import io.druid.data.input.ByteBufferInputRowParser;
+import io.druid.data.input.InputRow;
+
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.CoderResult;
+import java.nio.charset.CodingErrorAction;
+import java.util.Map;
+
+/**
+ */
+public class StringInputRowParser implements ByteBufferInputRowParser
+{
+ private static final Charset DEFAULT_CHARSET = Charsets.UTF_8;
+
+ private final ParseSpec parseSpec;
+ private final MapInputRowParser mapParser;
+ private final Parser parser;
+ private final Charset charset;
+
+ private CharBuffer chars = null;
+
+ @JsonCreator
+ public StringInputRowParser(
+ @JsonProperty("parseSpec") ParseSpec parseSpec,
+ @JsonProperty("encoding") String encoding
+ )
+ {
+ this.parseSpec = parseSpec;
+ this.mapParser = new MapInputRowParser(parseSpec);
+ this.parser = parseSpec.makeParser();
+
+ if (encoding != null) {
+ this.charset = Charset.forName(encoding);
+ } else {
+ this.charset = DEFAULT_CHARSET;
+ }
+ }
+
+ @Deprecated
+ public StringInputRowParser(ParseSpec parseSpec)
+ {
+ this(parseSpec, null);
+ }
+
+ @Override
+ public InputRow parse(ByteBuffer input)
+ {
+ return parseMap(buildStringKeyMap(input));
+ }
+
+ @JsonProperty
+ @Override
+ public ParseSpec getParseSpec()
+ {
+ return parseSpec;
+ }
+
+ @JsonProperty
+ public String getEncoding()
+ {
+ return charset.name();
+ }
+
+ @Override
+ public StringInputRowParser withParseSpec(ParseSpec parseSpec)
+ {
+ return new StringInputRowParser(parseSpec, getEncoding());
+ }
+
+ private Map buildStringKeyMap(ByteBuffer input)
+ {
+ int payloadSize = input.remaining();
+
+ if (chars == null || chars.remaining() < payloadSize) {
+ chars = CharBuffer.allocate(payloadSize);
+ }
+
+ final CoderResult coderResult = charset.newDecoder()
+ .onMalformedInput(CodingErrorAction.REPLACE)
+ .onUnmappableCharacter(CodingErrorAction.REPLACE)
+ .decode(input, chars, true);
+
+ Map theMap;
+ if (coderResult.isUnderflow()) {
+ chars.flip();
+ try {
+ theMap = parseString(chars.toString());
+ }
+ finally {
+ chars.clear();
+ }
+ } else {
+ throw new ParseException("Failed with CoderResult[%s]", coderResult);
+ }
+ return theMap;
+ }
+
+ private Map parseString(String inputString)
+ {
+ return parser.parse(inputString);
+ }
+
+ public InputRow parse(String input)
+ {
+ return parseMap(parseString(input));
+ }
+
+ private InputRow parseMap(Map theMap)
+ {
+ return mapParser.parse(theMap);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java b/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java
new file mode 100644
index 00000000000..adbf58b7649
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java
@@ -0,0 +1,78 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.metamx.common.parsers.Parser;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ */
+public class TimeAndDimsParseSpec extends ParseSpec
+{
+ @JsonCreator
+ public TimeAndDimsParseSpec(
+ @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
+ @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec
+ )
+ {
+ super(
+ timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null),
+ dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null)
+ );
+ }
+
+ public Parser makeParser()
+ {
+ return new Parser()
+ {
+ @Override
+ public Map parse(String input)
+ {
+ throw new UnsupportedOperationException("not supported");
+ }
+
+ @Override
+ public void setFieldNames(Iterable fieldNames)
+ {
+ throw new UnsupportedOperationException("not supported");
+ }
+
+ @Override
+ public List getFieldNames()
+ {
+ throw new UnsupportedOperationException("not supported");
+ }
+ };
+ }
+
+ public ParseSpec withTimestampSpec(TimestampSpec spec)
+ {
+ return new TimeAndDimsParseSpec(spec, getDimensionsSpec());
+ }
+
+ public ParseSpec withDimensionsSpec(DimensionsSpec spec)
+ {
+ return new TimeAndDimsParseSpec(getTimestampSpec(), spec);
+ }
+}
diff --git a/api/src/main/java/io/druid/data/input/impl/TimestampSpec.java b/api/src/main/java/io/druid/data/input/impl/TimestampSpec.java
new file mode 100644
index 00000000000..73340760041
--- /dev/null
+++ b/api/src/main/java/io/druid/data/input/impl/TimestampSpec.java
@@ -0,0 +1,114 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright 2012 - 2015 Metamarkets Group Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.druid.data.input.impl;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Function;
+import com.metamx.common.parsers.ParserUtils;
+import com.metamx.common.parsers.TimestampParser;
+import org.joda.time.DateTime;
+
+import java.util.Map;
+
+/**
+ */
+public class TimestampSpec
+{
+ private static final String DEFAULT_COLUMN = "timestamp";
+ private static final String DEFAULT_FORMAT = "auto";
+ private static final DateTime DEFAULT_MISSING_VALUE = null;
+
+ private final String timestampColumn;
+ private final String timestampFormat;
+ private final Function timestampConverter;
+ // this value should never be set for production data
+ private final DateTime missingValue;
+
+ @JsonCreator
+ public TimestampSpec(
+ @JsonProperty("column") String timestampColumn,
+ @JsonProperty("format") String format,
+ // this value should never be set for production data
+ @JsonProperty("missingValue") DateTime missingValue
+ )
+ {
+ this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn;
+ this.timestampFormat = format == null ? DEFAULT_FORMAT : format;
+ this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat);
+ this.missingValue = missingValue == null
+ ? DEFAULT_MISSING_VALUE
+ : missingValue;
+ }
+
+ @JsonProperty("column")
+ public String getTimestampColumn()
+ {
+ return timestampColumn;
+ }
+
+ @JsonProperty("format")
+ public String getTimestampFormat()
+ {
+ return timestampFormat;
+ }
+
+ @JsonProperty("missingValue")
+ public DateTime getMissingValue()
+ {
+ return missingValue;
+ }
+
+ public DateTime extractTimestamp(Map input)
+ {
+ final Object o = input.get(timestampColumn);
+
+ return o == null ? missingValue : timestampConverter.apply(o);
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ TimestampSpec that = (TimestampSpec) o;
+
+ if (!timestampColumn.equals(that.timestampColumn)) {
+ return false;
+ }
+ if (!timestampFormat.equals(that.timestampFormat)) {
+ return false;
+ }
+ return !(missingValue != null ? !missingValue.equals(that.missingValue) : that.missingValue != null);
+
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = timestampColumn.hashCode();
+ result = 31 * result + timestampFormat.hashCode();
+ result = 31 * result + (missingValue != null ? missingValue.hashCode() : 0);
+ return result;
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/Binders.java b/api/src/main/java/io/druid/guice/Binders.java
new file mode 100644
index 00000000000..b917974ac2b
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/Binders.java
@@ -0,0 +1,71 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Key;
+import com.google.inject.multibindings.MapBinder;
+import io.druid.segment.loading.DataSegmentArchiver;
+import io.druid.segment.loading.DataSegmentFinder;
+import io.druid.segment.loading.DataSegmentMover;
+import io.druid.segment.loading.DataSegmentKiller;
+import io.druid.segment.loading.DataSegmentPuller;
+import io.druid.segment.loading.DataSegmentPusher;
+import io.druid.tasklogs.TaskLogs;
+
+/**
+ */
+public class Binders
+{
+ public static MapBinder dataSegmentPullerBinder(Binder binder)
+ {
+ return MapBinder.newMapBinder(binder, String.class, DataSegmentPuller.class);
+ }
+
+ public static MapBinder dataSegmentKillerBinder(Binder binder)
+ {
+ return MapBinder.newMapBinder(binder, String.class, DataSegmentKiller.class);
+ }
+
+ public static MapBinder dataSegmentMoverBinder(Binder binder)
+ {
+ return MapBinder.newMapBinder(binder, String.class, DataSegmentMover.class);
+ }
+
+ public static MapBinder dataSegmentArchiverBinder(Binder binder)
+ {
+ return MapBinder.newMapBinder(binder, String.class, DataSegmentArchiver.class);
+ }
+
+ public static MapBinder dataSegmentPusherBinder(Binder binder)
+ {
+ return PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class));
+ }
+
+ public static MapBinder dataSegmentFinderBinder(Binder binder)
+ {
+ return PolyBind.optionBinder(binder, Key.get(DataSegmentFinder.class));
+ }
+
+ public static MapBinder taskLogsBinder(Binder binder)
+ {
+ return PolyBind.optionBinder(binder, Key.get(TaskLogs.class));
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/DruidGuiceExtensions.java b/api/src/main/java/io/druid/guice/DruidGuiceExtensions.java
new file mode 100644
index 00000000000..18d54029c11
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/DruidGuiceExtensions.java
@@ -0,0 +1,34 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Module;
+
+/**
+ */
+public class DruidGuiceExtensions implements Module
+{
+ @Override
+ public void configure(Binder binder)
+ {
+ binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON);
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/DruidScopes.java b/api/src/main/java/io/druid/guice/DruidScopes.java
new file mode 100644
index 00000000000..c8bfbdfd36b
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/DruidScopes.java
@@ -0,0 +1,45 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Key;
+import com.google.inject.Provider;
+import com.google.inject.Scope;
+import com.google.inject.Scopes;
+
+/**
+ */
+public class DruidScopes
+{
+ public static final Scope SINGLETON = new Scope()
+ {
+ @Override
+ public Provider scope(Key key, Provider unscoped)
+ {
+ return Scopes.SINGLETON.scope(key, unscoped);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "DruidScopes.SINGLETON";
+ }
+ };
+}
diff --git a/api/src/main/java/io/druid/guice/Jerseys.java b/api/src/main/java/io/druid/guice/Jerseys.java
new file mode 100644
index 00000000000..087c8d148f5
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/Jerseys.java
@@ -0,0 +1,37 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.TypeLiteral;
+import com.google.inject.multibindings.Multibinder;
+import io.druid.guice.annotations.JSR311Resource;
+
+/**
+ */
+public class Jerseys
+{
+ public static void addResource(Binder binder, Class> resourceClazz)
+ {
+ Multibinder.newSetBinder(binder, new TypeLiteral>(){}, JSR311Resource.class)
+ .addBinding()
+ .toInstance(resourceClazz);
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/JsonConfigProvider.java b/api/src/main/java/io/druid/guice/JsonConfigProvider.java
new file mode 100644
index 00000000000..54b64b6ecdf
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/JsonConfigProvider.java
@@ -0,0 +1,213 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+import com.google.inject.Binder;
+import com.google.inject.Inject;
+import com.google.inject.Key;
+import com.google.inject.Provider;
+import com.google.inject.util.Types;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.ParameterizedType;
+import java.util.Properties;
+
+
+/**
+ * Provides a singleton value of type {@code } from {@code Properties} bound in guice.
+ *
+ * Usage
+ * To install this provider, bind it in your guice module, like below.
+ *
+ *
+ * JsonConfigProvider.bind(binder, "druid.server", DruidServerConfig.class);
+ *
+ *
+ * In the above case, {@code druid.server} should be a key found in the {@code Properties} bound elsewhere.
+ * The value of that key should directly relate to the fields in {@code DruidServerConfig.class}.
+ *
+ * Implementation
+ *
+ * The state of {@code } is defined by the value of the property {@code propertyBase}.
+ * This value is a json structure, decoded via {@link JsonConfigurator#configurate(java.util.Properties, String, Class)}.
+ *
+ *
+ * An example might be if DruidServerConfig.class were
+ *
+ *
+ * public class DruidServerConfig
+ * {
+ * @JsonProperty @NotNull public String hostname = null;
+ * @JsonProperty @Min(1025) public int port = 8080;
+ * }
+ *
+ *
+ * And your Properties object had in it
+ *
+ *
+ * druid.server.hostname=0.0.0.0
+ * druid.server.port=3333
+ *
+ *
+ * Then this would bind a singleton instance of a DruidServerConfig object with hostname = "0.0.0.0" and port = 3333.
+ *
+ * If the port weren't set in the properties, then the default of 8080 would be taken. Essentially, it is the same as
+ * subtracting the "druid.server" prefix from the properties and building a Map which is then passed into
+ * ObjectMapper.convertValue()
+ *
+ * @param type of config object to provide.
+ */
+public class JsonConfigProvider implements Provider>
+{
+ @SuppressWarnings("unchecked")
+ public static void bind(Binder binder, String propertyBase, Class classToProvide)
+ {
+ bind(
+ binder,
+ propertyBase,
+ classToProvide,
+ Key.get(classToProvide),
+ (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide))
+ );
+ }
+
+ @SuppressWarnings("unchecked")
+ public static void bind(Binder binder, String propertyBase, Class classToProvide, Annotation annotation)
+ {
+ bind(
+ binder,
+ propertyBase,
+ classToProvide,
+ Key.get(classToProvide, annotation),
+ (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation)
+ );
+ }
+
+ @SuppressWarnings("unchecked")
+ public static void bind(
+ Binder binder,
+ String propertyBase,
+ Class classToProvide,
+ Class extends Annotation> annotation
+ )
+ {
+ bind(
+ binder,
+ propertyBase,
+ classToProvide,
+ Key.get(classToProvide, annotation),
+ (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation)
+ );
+ }
+
+ @SuppressWarnings("unchecked")
+ public static void bind(
+ Binder binder,
+ String propertyBase,
+ Class clazz,
+ Key instanceKey,
+ Key> supplierKey
+ )
+ {
+ binder.bind(supplierKey).toProvider((Provider) of(propertyBase, clazz)).in(LazySingleton.class);
+ binder.bind(instanceKey).toProvider(new SupplierProvider(supplierKey));
+ }
+
+ @SuppressWarnings("unchecked")
+ public static void bindInstance(
+ Binder binder,
+ Key bindKey,
+ T instance
+ )
+ {
+ binder.bind(bindKey).toInstance(instance);
+
+ final ParameterizedType supType = Types.newParameterizedType(Supplier.class, bindKey.getTypeLiteral().getType());
+ final Key supplierKey;
+
+ if (bindKey.getAnnotationType() != null) {
+ supplierKey = Key.get(supType, bindKey.getAnnotationType());
+ }
+ else if (bindKey.getAnnotation() != null) {
+ supplierKey = Key.get(supType, bindKey.getAnnotation());
+ }
+ else {
+ supplierKey = Key.get(supType);
+ }
+
+ binder.bind(supplierKey).toInstance(Suppliers.ofInstance(instance));
+ }
+
+ public static JsonConfigProvider of(String propertyBase, Class classToProvide)
+ {
+ return new JsonConfigProvider(propertyBase, classToProvide);
+ }
+
+ private final String propertyBase;
+ private final Class classToProvide;
+
+ private Properties props;
+ private JsonConfigurator configurator;
+
+ private Supplier retVal = null;
+
+ public JsonConfigProvider(
+ String propertyBase,
+ Class classToProvide
+ )
+ {
+ this.propertyBase = propertyBase;
+ this.classToProvide = classToProvide;
+ }
+
+ @Inject
+ public void inject(
+ Properties props,
+ JsonConfigurator configurator
+ )
+ {
+ this.props = props;
+ this.configurator = configurator;
+ }
+
+ @Override
+ public Supplier get()
+ {
+ if (retVal != null) {
+ return retVal;
+ }
+
+ try {
+ final T config = configurator.configurate(props, propertyBase, classToProvide);
+ retVal = Suppliers.ofInstance(config);
+ }
+ catch (RuntimeException e) {
+ // When a runtime exception gets thrown out, this provider will get called again if the object is asked for again.
+ // This will have the same failed result, 'cause when it's called no parameters will have actually changed.
+ // Guice will then report the same error multiple times, which is pretty annoying. Cache a null supplier and
+ // return that instead. This is technically enforcing a singleton, but such is life.
+ retVal = Suppliers.ofInstance(null);
+ throw e;
+ }
+ return retVal;
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/JsonConfigurator.java b/api/src/main/java/io/druid/guice/JsonConfigurator.java
new file mode 100644
index 00000000000..3a5050749a6
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/JsonConfigurator.java
@@ -0,0 +1,184 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.introspect.AnnotatedField;
+import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition;
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.inject.Inject;
+import com.google.inject.ProvisionException;
+import com.google.inject.spi.Message;
+import com.metamx.common.logger.Logger;
+
+import javax.validation.ConstraintViolation;
+import javax.validation.ElementKind;
+import javax.validation.Path;
+import javax.validation.Validator;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ */
+public class JsonConfigurator
+{
+ private static final Logger log = new Logger(JsonConfigurator.class);
+
+ private final ObjectMapper jsonMapper;
+ private final Validator validator;
+
+ @Inject
+ public JsonConfigurator(
+ ObjectMapper jsonMapper,
+ Validator validator
+ )
+ {
+ this.jsonMapper = jsonMapper;
+ this.validator = validator;
+ }
+
+ public T configurate(Properties props, String propertyPrefix, Class clazz) throws ProvisionException
+ {
+ verifyClazzIsConfigurable(clazz);
+
+ // Make it end with a period so we only include properties with sub-object thingies.
+ final String propertyBase = propertyPrefix.endsWith(".") ? propertyPrefix : propertyPrefix + ".";
+
+ Map jsonMap = Maps.newHashMap();
+ for (String prop : props.stringPropertyNames()) {
+ if (prop.startsWith(propertyBase)) {
+ final String propValue = props.getProperty(prop);
+ Object value;
+ try {
+ // If it's a String Jackson wants it to be quoted, so check if it's not an object or array and quote.
+ String modifiedPropValue = propValue;
+ if (! (modifiedPropValue.startsWith("[") || modifiedPropValue.startsWith("{"))) {
+ modifiedPropValue = jsonMapper.writeValueAsString(propValue);
+ }
+ value = jsonMapper.readValue(modifiedPropValue, Object.class);
+ }
+ catch (IOException e) {
+ log.info(e, "Unable to parse [%s]=[%s] as a json object, using as is.", prop, propValue);
+ value = propValue;
+ }
+
+ jsonMap.put(prop.substring(propertyBase.length()), value);
+ }
+ }
+
+ final T config;
+ try {
+ config = jsonMapper.convertValue(jsonMap, clazz);
+ }
+ catch (IllegalArgumentException e) {
+ throw new ProvisionException(
+ String.format("Problem parsing object at prefix[%s]: %s.", propertyPrefix, e.getMessage()), e
+ );
+ }
+
+ final Set> violations = validator.validate(config);
+ if (!violations.isEmpty()) {
+ List messages = Lists.newArrayList();
+
+ for (ConstraintViolation violation : violations) {
+ String path = "";
+ try {
+ Class> beanClazz = violation.getRootBeanClass();
+ final Iterator iter = violation.getPropertyPath().iterator();
+ while (iter.hasNext()) {
+ Path.Node next = iter.next();
+ if (next.getKind() == ElementKind.PROPERTY) {
+ final String fieldName = next.getName();
+ final Field theField = beanClazz.getDeclaredField(fieldName);
+
+ if (theField.getAnnotation(JacksonInject.class) != null) {
+ path = String.format(" -- Injected field[%s] not bound!?", fieldName);
+ break;
+ }
+
+ JsonProperty annotation = theField.getAnnotation(JsonProperty.class);
+ final boolean noAnnotationValue = annotation == null || Strings.isNullOrEmpty(annotation.value());
+ final String pathPart = noAnnotationValue ? fieldName : annotation.value();
+ if (path.isEmpty()) {
+ path += pathPart;
+ }
+ else {
+ path += "." + pathPart;
+ }
+ }
+ }
+ }
+ catch (NoSuchFieldException e) {
+ throw Throwables.propagate(e);
+ }
+
+ messages.add(String.format("%s - %s", path, violation.getMessage()));
+ }
+
+ throw new ProvisionException(
+ Iterables.transform(
+ messages,
+ new Function()
+ {
+ @Override
+ public Message apply(String input)
+ {
+ return new Message(String.format("%s%s", propertyBase, input));
+ }
+ }
+ )
+ );
+ }
+
+ log.info("Loaded class[%s] from props[%s] as [%s]", clazz, propertyBase, config);
+
+ return config;
+ }
+
+ private void verifyClazzIsConfigurable(Class clazz)
+ {
+ final List beanDefs = jsonMapper.getSerializationConfig()
+ .introspect(jsonMapper.constructType(clazz))
+ .findProperties();
+ for (BeanPropertyDefinition beanDef : beanDefs) {
+ final AnnotatedField field = beanDef.getField();
+ if (field == null || !field.hasAnnotation(JsonProperty.class)) {
+ throw new ProvisionException(
+ String.format(
+ "JsonConfigurator requires Jackson-annotated Config objects to have field annotations. %s doesn't",
+ clazz
+ )
+ );
+ }
+ }
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/KeyHolder.java b/api/src/main/java/io/druid/guice/KeyHolder.java
new file mode 100644
index 00000000000..2a996f5cc50
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/KeyHolder.java
@@ -0,0 +1,41 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Key;
+
+/**
+ */
+public class KeyHolder
+{
+ private final Key extends T> key;
+
+ public KeyHolder(
+ Key extends T> key
+ )
+ {
+ this.key = key;
+ }
+
+ public Key extends T> getKey()
+ {
+ return key;
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/LazySingleton.java b/api/src/main/java/io/druid/guice/LazySingleton.java
new file mode 100644
index 00000000000..b9c188d2cfe
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/LazySingleton.java
@@ -0,0 +1,37 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.ScopeAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+/**
+ */
+@Target({ ElementType.TYPE, ElementType.METHOD })
+@Retention(RUNTIME)
+@ScopeAnnotation
+public @interface LazySingleton
+{
+}
diff --git a/api/src/main/java/io/druid/guice/LifecycleModule.java b/api/src/main/java/io/druid/guice/LifecycleModule.java
new file mode 100644
index 00000000000..2b8473fa65a
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/LifecycleModule.java
@@ -0,0 +1,163 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import com.google.inject.TypeLiteral;
+import com.google.inject.multibindings.Multibinder;
+import com.google.inject.name.Names;
+import com.metamx.common.lifecycle.Lifecycle;
+
+import java.lang.annotation.Annotation;
+import java.util.Set;
+
+/**
+ * A Module to add lifecycle management to the injector. {@link DruidGuiceExtensions} must also be included.
+ */
+public class LifecycleModule implements Module
+{
+ private final LifecycleScope scope = new LifecycleScope(Lifecycle.Stage.NORMAL);
+ private final LifecycleScope lastScope = new LifecycleScope(Lifecycle.Stage.LAST);
+
+ /**
+ * Registers a class to instantiate eagerly. Classes mentioned here will be pulled out of
+ * the injector with an injector.getInstance() call when the lifecycle is created.
+ *
+ * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
+ * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
+ * side-effect even if nothing actually directly depends on the instance.
+ *
+ * This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point from the injector, not to
+ * auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is also possible to just bind things eagerly with Guice,
+ * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules
+ * is less error-prone.
+ *
+ * @param clazz, the class to instantiate
+ * @return this, for chaining.
+ */
+ public static void register(Binder binder, Class> clazz)
+ {
+ registerKey(binder, Key.get(clazz));
+ }
+
+ /**
+ * Registers a class/annotation combination to instantiate eagerly. Classes mentioned here will be pulled out of
+ * the injector with an injector.getInstance() call when the lifecycle is created.
+ *
+ * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
+ * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
+ * side-effect even if nothing actually directly depends on the instance.
+ *
+ * This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point from the injector, not to
+ * auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is also possible to just bind things eagerly with Guice,
+ * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules
+ * is less error-prone.
+ *
+ * @param clazz, the class to instantiate
+ * @param annotation The annotation instance to register with Guice, usually a Named annotation
+ * @return this, for chaining.
+ */
+ public static void register(Binder binder, Class> clazz, Annotation annotation)
+ {
+ registerKey(binder, Key.get(clazz, annotation));
+ }
+
+ /**
+ * Registers a class/annotation combination to instantiate eagerly. Classes mentioned here will be pulled out of
+ * the injector with an injector.getInstance() call when the lifecycle is created.
+ *
+ * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
+ * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
+ * side-effect even if nothing actually directly depends on the instance.
+ *
+ * This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point from the injector, not to
+ * auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is also possible to just bind things eagerly with Guice,
+ * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules
+ * is less error-prone.
+ *
+ * @param clazz, the class to instantiate
+ * @param annotation The annotation class to register with Guice
+ * @return this, for chaining
+ */
+ public static void register(Binder binder, Class> clazz, Class extends Annotation> annotation)
+ {
+ registerKey(binder, Key.get(clazz, annotation));
+ }
+
+ /**
+ * Registers a key to instantiate eagerly. {@link com.google.inject.Key}s mentioned here will be pulled out of
+ * the injector with an injector.getInstance() call when the lifecycle is created.
+ *
+ * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
+ * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
+ * side-effect even if nothing actually directly depends on the instance.
+ *
+ * This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point
+ * from the injector, not to auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is
+ * also possible to just bind things eagerly with Guice, it is not clear which is actually the best approach.
+ * This is more explicit, but eager bindings inside of modules is less error-prone.
+ *
+ * @param key The key to use in finding the DruidNode instance
+ */
+ public static void registerKey(Binder binder, Key> key)
+ {
+ getEagerBinder(binder).addBinding().toInstance(new KeyHolder(key));
+ }
+
+ private static Multibinder getEagerBinder(Binder binder)
+ {
+ return Multibinder.newSetBinder(binder, KeyHolder.class, Names.named("lifecycle"));
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ getEagerBinder(binder); // Load up the eager binder so that it will inject the empty set at a minimum.
+
+ binder.bindScope(ManageLifecycle.class, scope);
+ binder.bindScope(ManageLifecycleLast.class, lastScope);
+ }
+
+ @Provides @LazySingleton
+ public Lifecycle getLifecycle(final Injector injector)
+ {
+ final Key> keyHolderKey = Key.get(new TypeLiteral>(){}, Names.named("lifecycle"));
+ final Set eagerClasses = injector.getInstance(keyHolderKey);
+
+ Lifecycle lifecycle = new Lifecycle(){
+ @Override
+ public void start() throws Exception
+ {
+ for (KeyHolder> holder : eagerClasses) {
+ injector.getInstance(holder.getKey()); // Pull the key so as to "eagerly" load up the class.
+ }
+ super.start();
+ }
+ };
+ scope.setLifecycle(lifecycle);
+ lastScope.setLifecycle(lifecycle);
+
+ return lifecycle;
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/LifecycleScope.java b/api/src/main/java/io/druid/guice/LifecycleScope.java
new file mode 100644
index 00000000000..b4fb8df1f47
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/LifecycleScope.java
@@ -0,0 +1,92 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.common.collect.Lists;
+import com.google.inject.Key;
+import com.google.inject.Provider;
+import com.google.inject.Scope;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.common.logger.Logger;
+
+import java.util.List;
+
+/**
+ * A scope that adds objects to the Lifecycle. This is by definition also a lazy singleton scope.
+ */
+public class LifecycleScope implements Scope
+{
+ private static final Logger log = new Logger(LifecycleScope.class);
+ private final Lifecycle.Stage stage;
+
+ private Lifecycle lifecycle;
+ private List instances = Lists.newLinkedList();
+
+ public LifecycleScope(Lifecycle.Stage stage)
+ {
+ this.stage = stage;
+ }
+
+ public void setLifecycle(Lifecycle lifecycle)
+ {
+ this.lifecycle = lifecycle;
+ synchronized (instances) {
+ for (Object instance : instances) {
+ lifecycle.addManagedInstance(instance);
+ }
+ }
+ }
+
+ @Override
+ public Provider scope(final Key key, final Provider unscoped)
+ {
+ return new Provider()
+ {
+ private volatile T value = null;
+
+ @Override
+ public synchronized T get()
+ {
+ if (value == null) {
+ final T retVal = unscoped.get();
+
+ synchronized (instances) {
+ if (lifecycle == null) {
+ instances.add(retVal);
+ }
+ else {
+ try {
+ lifecycle.addMaybeStartManagedInstance(retVal, stage);
+ }
+ catch (Exception e) {
+ log.warn(e, "Caught exception when trying to create a[%s]", key);
+ return null;
+ }
+ }
+ }
+
+ value = retVal;
+ }
+
+ return value;
+ }
+ };
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/ManageLifecycle.java b/api/src/main/java/io/druid/guice/ManageLifecycle.java
new file mode 100644
index 00000000000..b4d90110d3f
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/ManageLifecycle.java
@@ -0,0 +1,40 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.ScopeAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+/**
+ * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle}
+ *
+ * This Scope gets defined by {@link io.druid.guice.LifecycleModule}
+ */
+@Target({ ElementType.TYPE, ElementType.METHOD })
+@Retention(RUNTIME)
+@ScopeAnnotation
+public @interface ManageLifecycle
+{
+}
diff --git a/api/src/main/java/io/druid/guice/ManageLifecycleLast.java b/api/src/main/java/io/druid/guice/ManageLifecycleLast.java
new file mode 100644
index 00000000000..f6feae9023e
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/ManageLifecycleLast.java
@@ -0,0 +1,40 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.ScopeAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+/**
+ * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} and set to be on Stage.LAST
+ *
+ * This Scope gets defined by {@link io.druid.guice.LifecycleModule}
+ */
+@Target({ ElementType.TYPE, ElementType.METHOD })
+@Retention(RUNTIME)
+@ScopeAnnotation
+public @interface ManageLifecycleLast
+{
+}
diff --git a/api/src/main/java/io/druid/guice/PolyBind.java b/api/src/main/java/io/druid/guice/PolyBind.java
new file mode 100644
index 00000000000..96d012f0f62
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/PolyBind.java
@@ -0,0 +1,184 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Provider;
+import com.google.inject.ProvisionException;
+import com.google.inject.TypeLiteral;
+import com.google.inject.binder.ScopedBindingBuilder;
+import com.google.inject.multibindings.MapBinder;
+import com.google.inject.util.Types;
+
+import java.lang.reflect.ParameterizedType;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Provides the ability to create "polymorphic" bindings. Where the polymorphism is actually just making a decision
+ * based on a value in a Properties.
+ *
+ * The workflow is that you first create a choice by calling createChoice(). Then you create options using the binder
+ * returned by the optionBinder() method. Multiple different modules can call optionBinder and all options will be
+ * reflected at injection time as long as equivalent interface Key objects are passed into the various methods.
+ */
+public class PolyBind
+{
+ /**
+ * Sets up a "choice" for the injector to resolve at injection time.
+ *
+ * @param binder the binder for the injector that is being configured
+ * @param property the property that will be checked to determine the implementation choice
+ * @param interfaceKey the interface that will be injected using this choice
+ * @param defaultKey the default instance to be injected if the property doesn't match a choice. Can be null
+ * @param interface type
+ * @return A ScopedBindingBuilder so that scopes can be added to the binding, if required.
+ */
+ public static ScopedBindingBuilder createChoice(
+ Binder binder,
+ String property,
+ Key interfaceKey,
+ Key extends T> defaultKey
+ )
+ {
+ return createChoiceWithDefault(binder, property, interfaceKey, defaultKey, null);
+ }
+
+ /**
+ * Sets up a "choice" for the injector to resolve at injection time.
+ *
+ * @param binder the binder for the injector that is being configured
+ * @param property the property that will be checked to determine the implementation choice
+ * @param interfaceKey the interface that will be injected using this choice
+ * @param defaultKey the default instance to be injected if the property doesn't match a choice. Can be null
+ * @param defaultPropertyValue the default property value to use if the property is not set.
+ * @param interface type
+ * @return A ScopedBindingBuilder so that scopes can be added to the binding, if required.
+ */
+ public static ScopedBindingBuilder createChoiceWithDefault(
+ Binder binder,
+ String property,
+ Key interfaceKey,
+ Key extends T> defaultKey,
+ String defaultPropertyValue
+ )
+ {
+ return binder.bind(interfaceKey).toProvider(new ConfiggedProvider(interfaceKey, property, defaultKey, defaultPropertyValue));
+ }
+
+ /**
+ * Binds an option for a specific choice. The choice must already be registered on the injector for this to work.
+ *
+ * @param binder the binder for the injector that is being configured
+ * @param interfaceKey the interface that will have an option added to it. This must equal the
+ * Key provided to createChoice
+ * @param interface type
+ * @return A MapBinder that can be used to create the actual option bindings.
+ */
+ public static MapBinder optionBinder(Binder binder, Key interfaceKey)
+ {
+ final TypeLiteral interfaceType = interfaceKey.getTypeLiteral();
+
+ if (interfaceKey.getAnnotation() != null) {
+ return MapBinder.newMapBinder(
+ binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation()
+ );
+ }
+ else if (interfaceKey.getAnnotationType() != null) {
+ return MapBinder.newMapBinder(
+ binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotationType()
+ );
+ }
+ else {
+ return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType);
+ }
+ }
+
+ static class ConfiggedProvider implements Provider
+ {
+ private final Key key;
+ private final String property;
+ private final Key extends T> defaultKey;
+ private final String defaultPropertyValue;
+
+ private Injector injector;
+ private Properties props;
+
+ ConfiggedProvider(
+ Key key,
+ String property,
+ Key extends T> defaultKey,
+ String defaultPropertyValue
+ )
+ {
+ this.key = key;
+ this.property = property;
+ this.defaultKey = defaultKey;
+ this.defaultPropertyValue = defaultPropertyValue;
+ }
+
+ @Inject
+ void configure(Injector injector, Properties props)
+ {
+ this.injector = injector;
+ this.props = props;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public T get()
+ {
+ final ParameterizedType mapType = Types.mapOf(
+ String.class, Types.newParameterizedType(Provider.class, key.getTypeLiteral().getType())
+ );
+
+ final Map> implsMap;
+ if (key.getAnnotation() != null) {
+ implsMap = (Map>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
+ }
+ else if (key.getAnnotationType() != null) {
+ implsMap = (Map>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
+ }
+ else {
+ implsMap = (Map>) injector.getInstance(Key.get(mapType));
+ }
+
+ String implName = props.getProperty(property);
+ if (implName == null) {
+ implName = defaultPropertyValue;
+ }
+ final Provider provider = implsMap.get(implName);
+
+ if (provider == null) {
+ if (defaultKey == null) {
+ throw new ProvisionException(
+ String.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet())
+ );
+ }
+ return injector.getInstance(defaultKey);
+ }
+
+ return provider.get();
+ }
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/SupplierProvider.java b/api/src/main/java/io/druid/guice/SupplierProvider.java
new file mode 100644
index 00000000000..b545fce56c7
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/SupplierProvider.java
@@ -0,0 +1,54 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice;
+
+import com.google.common.base.Supplier;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import com.google.inject.Key;
+import com.google.inject.Provider;
+
+/**
+ */
+public class SupplierProvider implements Provider
+{
+ private final Key> supplierKey;
+
+ private Provider> supplierProvider;
+
+ public SupplierProvider(
+ Key> supplierKey
+ )
+ {
+ this.supplierKey = supplierKey;
+ }
+
+ @Inject
+ public void configure(Injector injector)
+ {
+ this.supplierProvider = injector.getProvider(supplierKey);
+ }
+
+ @Override
+ public T get()
+ {
+ return supplierProvider.get().get();
+ }
+}
diff --git a/api/src/main/java/io/druid/guice/annotations/Global.java b/api/src/main/java/io/druid/guice/annotations/Global.java
new file mode 100644
index 00000000000..579574a9a66
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/annotations/Global.java
@@ -0,0 +1,36 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ */
+@BindingAnnotation
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Global
+{
+}
diff --git a/api/src/main/java/io/druid/guice/annotations/JSR311Resource.java b/api/src/main/java/io/druid/guice/annotations/JSR311Resource.java
new file mode 100644
index 00000000000..eec8ee3f986
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/annotations/JSR311Resource.java
@@ -0,0 +1,36 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ */
+@BindingAnnotation
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface JSR311Resource
+{
+}
diff --git a/api/src/main/java/io/druid/guice/annotations/Json.java b/api/src/main/java/io/druid/guice/annotations/Json.java
new file mode 100644
index 00000000000..136ca516182
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/annotations/Json.java
@@ -0,0 +1,36 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+public @interface Json
+{
+}
diff --git a/api/src/main/java/io/druid/guice/annotations/Self.java b/api/src/main/java/io/druid/guice/annotations/Self.java
new file mode 100644
index 00000000000..a4f68d86f0c
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/annotations/Self.java
@@ -0,0 +1,36 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+public @interface Self
+{
+}
diff --git a/api/src/main/java/io/druid/guice/annotations/Smile.java b/api/src/main/java/io/druid/guice/annotations/Smile.java
new file mode 100644
index 00000000000..28227c20324
--- /dev/null
+++ b/api/src/main/java/io/druid/guice/annotations/Smile.java
@@ -0,0 +1,36 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+public @interface Smile
+{
+}
diff --git a/api/src/main/java/io/druid/initialization/DruidModule.java b/api/src/main/java/io/druid/initialization/DruidModule.java
new file mode 100644
index 00000000000..eb8fcfaec12
--- /dev/null
+++ b/api/src/main/java/io/druid/initialization/DruidModule.java
@@ -0,0 +1,31 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.initialization;
+
+import com.fasterxml.jackson.databind.Module;
+
+import java.util.List;
+
+/**
+ */
+public interface DruidModule extends com.google.inject.Module
+{
+ public List extends Module> getJacksonModules();
+}
diff --git a/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java b/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java
new file mode 100644
index 00000000000..46eeff5ef6a
--- /dev/null
+++ b/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java
@@ -0,0 +1,46 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.jackson;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ */
+public class CommaListJoinDeserializer extends StdScalarDeserializer>
+{
+ protected CommaListJoinDeserializer()
+ {
+ super(List.class);
+ }
+
+ @Override
+ public List deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
+ throws IOException, JsonProcessingException
+ {
+ return Arrays.asList(jsonParser.getText().split(","));
+ }
+}
diff --git a/api/src/main/java/io/druid/jackson/CommaListJoinSerializer.java b/api/src/main/java/io/druid/jackson/CommaListJoinSerializer.java
new file mode 100644
index 00000000000..bb09cdb166a
--- /dev/null
+++ b/api/src/main/java/io/druid/jackson/CommaListJoinSerializer.java
@@ -0,0 +1,48 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.jackson;
+
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer;
+import com.google.common.base.Joiner;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ */
+public class CommaListJoinSerializer extends StdScalarSerializer>
+{
+ private static final Joiner joiner = Joiner.on(",");
+
+ protected CommaListJoinSerializer()
+ {
+ super(List.class, true);
+ }
+
+ @Override
+ public void serialize(List value, JsonGenerator jgen, SerializerProvider provider)
+ throws IOException, JsonGenerationException
+ {
+ jgen.writeString(joiner.join(value));
+ }
+}
diff --git a/api/src/main/java/io/druid/query/SegmentDescriptor.java b/api/src/main/java/io/druid/query/SegmentDescriptor.java
new file mode 100644
index 00000000000..cfad1e08298
--- /dev/null
+++ b/api/src/main/java/io/druid/query/SegmentDescriptor.java
@@ -0,0 +1,107 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.query;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.joda.time.Interval;
+
+/**
+*/
+public class SegmentDescriptor
+{
+ private final Interval interval;
+ private final String version;
+ private final int partitionNumber;
+
+ @JsonCreator
+ public SegmentDescriptor(
+ @JsonProperty("itvl") Interval interval,
+ @JsonProperty("ver") String version,
+ @JsonProperty("part") int partitionNumber
+ )
+ {
+ this.interval = interval;
+ this.version = version;
+ this.partitionNumber = partitionNumber;
+ }
+
+ @JsonProperty("itvl")
+ public Interval getInterval()
+ {
+ return interval;
+ }
+
+ @JsonProperty("ver")
+ public String getVersion()
+ {
+ return version;
+ }
+
+ @JsonProperty("part")
+ public int getPartitionNumber()
+ {
+ return partitionNumber;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ SegmentDescriptor that = (SegmentDescriptor) o;
+
+ if (partitionNumber != that.partitionNumber) {
+ return false;
+ }
+ if (interval != null ? !interval.equals(that.interval) : that.interval != null) {
+ return false;
+ }
+ if (version != null ? !version.equals(that.version) : that.version != null) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = interval != null ? interval.hashCode() : 0;
+ result = 31 * result + (version != null ? version.hashCode() : 0);
+ result = 31 * result + partitionNumber;
+ return result;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "SegmentDescriptor{" +
+ "interval=" + interval +
+ ", version='" + version + '\'' +
+ ", partitionNumber=" + partitionNumber +
+ '}';
+ }
+}
diff --git a/api/src/main/java/io/druid/segment/SegmentUtils.java b/api/src/main/java/io/druid/segment/SegmentUtils.java
new file mode 100644
index 00000000000..32f7cb650c9
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/SegmentUtils.java
@@ -0,0 +1,48 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment;
+
+import com.google.common.io.Files;
+import com.google.common.primitives.Ints;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ */
+public class SegmentUtils
+{
+ public static int getVersionFromDir(File inDir) throws IOException
+ {
+ File versionFile = new File(inDir, "version.bin");
+ if (versionFile.exists()) {
+ return Ints.fromByteArray(Files.toByteArray(versionFile));
+ }
+
+ final File indexFile = new File(inDir, "index.drd");
+ int version;
+ try (InputStream in = new FileInputStream(indexFile)) {
+ version = in.read();
+ }
+ return version;
+ }
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentArchiver.java b/api/src/main/java/io/druid/segment/loading/DataSegmentArchiver.java
new file mode 100644
index 00000000000..869f0591dd6
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentArchiver.java
@@ -0,0 +1,28 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import io.druid.timeline.DataSegment;
+
+public interface DataSegmentArchiver
+{
+ public DataSegment archive(DataSegment segment) throws SegmentLoadingException;
+ public DataSegment restore(DataSegment segment) throws SegmentLoadingException;
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java b/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java
new file mode 100644
index 00000000000..60a747d8985
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentFinder.java
@@ -0,0 +1,46 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import io.druid.timeline.DataSegment;
+
+import java.util.Set;
+
+/**
+ * A DataSegmentFinder is responsible for finding Druid segments underneath a specified directory and optionally updates
+ * all descriptor.json files on deep storage with correct loadSpec.
+ */
+public interface DataSegmentFinder
+{
+ /**
+ * This method should first recursively look for descriptor.json underneath workingDirPath and then verify that
+ * index.zip exists in the same folder. If not, it should throw SegmentLoadingException to let the caller know that
+ * descriptor.json exists while index.zip doesn't. If a segment is found and updateDescriptor is set, then this method
+ * should update the loadSpec in descriptor.json to reflect the location from where it was found. After the search,
+ * this method should return the set of segments that were found.
+ *
+ * @param workingDirPath the String representation of the working directory path
+ * @param updateDescriptor if true, update loadSpec in descriptor.json if loadSpec's location is different from where
+ * desciptor.json was found
+ *
+ * @return a set of segments that were found underneath workingDirPath
+ */
+ Set findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException;
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java b/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java
new file mode 100644
index 00000000000..16f8f234865
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentKiller.java
@@ -0,0 +1,29 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import io.druid.timeline.DataSegment;
+
+/**
+ */
+public interface DataSegmentKiller
+{
+ public void kill(DataSegment segments) throws SegmentLoadingException;
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentMover.java b/api/src/main/java/io/druid/segment/loading/DataSegmentMover.java
new file mode 100644
index 00000000000..0ff92de61b6
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentMover.java
@@ -0,0 +1,29 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import io.druid.timeline.DataSegment;
+
+import java.util.Map;
+
+public interface DataSegmentMover
+{
+ public DataSegment move(DataSegment segment, Map targetLoadSpec) throws SegmentLoadingException;
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentPuller.java b/api/src/main/java/io/druid/segment/loading/DataSegmentPuller.java
new file mode 100644
index 00000000000..9b83de147df
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentPuller.java
@@ -0,0 +1,40 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import io.druid.timeline.DataSegment;
+
+import java.io.File;
+
+/**
+ * A DataSegmentPuller is responsible for pulling data for a particular segment into a particular directory
+ */
+public interface DataSegmentPuller
+{
+ /**
+ * Pull down segment files for the given DataSegment and put them in the given directory.
+ *
+ * @param segment The segment to pull down files for
+ * @param dir The directory to store the files in
+ *
+ * @throws SegmentLoadingException if there are any errors
+ */
+ public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException;
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java b/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java
new file mode 100644
index 00000000000..aeddcb3ea8b
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentPusher.java
@@ -0,0 +1,31 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import io.druid.timeline.DataSegment;
+
+import java.io.File;
+import java.io.IOException;
+
+public interface DataSegmentPusher
+{
+ public String getPathForHadoop(String dataSource);
+ public DataSegment push(File file, DataSegment segment) throws IOException;
+}
diff --git a/api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java b/api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java
new file mode 100644
index 00000000000..54f1d51afcc
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java
@@ -0,0 +1,67 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import com.google.common.base.Joiner;
+import io.druid.timeline.DataSegment;
+import org.joda.time.format.ISODateTimeFormat;
+
+/**
+ */
+public class DataSegmentPusherUtil
+{
+ private static final Joiner JOINER = Joiner.on("/").skipNulls();
+
+ // Note: storage directory structure format = .../dataSource/interval/version/partitionNumber/
+ // If above format is ever changed, make sure to change it appropriately in other places
+ // e.g. HDFSDataSegmentKiller uses this information to clean the version, interval and dataSource directories
+ // on segment deletion if segment being deleted was the only segment
+ public static String getStorageDir(DataSegment segment)
+ {
+ return JOINER.join(
+ segment.getDataSource(),
+ String.format(
+ "%s_%s",
+ segment.getInterval().getStart(),
+ segment.getInterval().getEnd()
+ ),
+ segment.getVersion(),
+ segment.getShardSpec().getPartitionNum()
+ );
+ }
+
+ /**
+ * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in
+ * path names. So we format paths differently for HDFS.
+ */
+ public static String getHdfsStorageDir(DataSegment segment)
+ {
+ return JOINER.join(
+ segment.getDataSource(),
+ String.format(
+ "%s_%s",
+ segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()),
+ segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime())
+ ),
+ segment.getVersion().replaceAll(":", "_"),
+ segment.getShardSpec().getPartitionNum()
+ );
+ }
+}
diff --git a/api/src/main/java/io/druid/segment/loading/LoadSpec.java b/api/src/main/java/io/druid/segment/loading/LoadSpec.java
new file mode 100644
index 00000000000..5e5edf3fb96
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/LoadSpec.java
@@ -0,0 +1,49 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+
+import java.io.File;
+
+/**
+ * A means of pulling segment files into a destination directory
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+public interface LoadSpec
+{
+ /**
+ * Method should put the segment files in the directory passed
+ * @param destDir The destination directory
+ * @return The byte count of data put in the destination directory
+ */
+ public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException;
+
+ // Hold interesting data about the results of the segment load
+ public static class LoadSpecResult{
+ private final long size;
+ public LoadSpecResult(long size){
+ this.size = size;
+ }
+ public long getSize(){
+ return this.size;
+ }
+ }
+}
diff --git a/api/src/main/java/io/druid/segment/loading/SegmentLoadingException.java b/api/src/main/java/io/druid/segment/loading/SegmentLoadingException.java
new file mode 100644
index 00000000000..9cbbbd21f71
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/SegmentLoadingException.java
@@ -0,0 +1,42 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+/**
+ */
+public class SegmentLoadingException extends Exception
+{
+ public SegmentLoadingException(
+ String formatString,
+ Object... objs
+ )
+ {
+ super(String.format(formatString, objs));
+ }
+
+ public SegmentLoadingException(
+ Throwable cause,
+ String formatString,
+ Object... objs
+ )
+ {
+ super(String.format(formatString, objs), cause);
+ }
+}
diff --git a/api/src/main/java/io/druid/segment/loading/URIDataPuller.java b/api/src/main/java/io/druid/segment/loading/URIDataPuller.java
new file mode 100644
index 00000000000..c81222ded61
--- /dev/null
+++ b/api/src/main/java/io/druid/segment/loading/URIDataPuller.java
@@ -0,0 +1,62 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.segment.loading;
+
+import com.google.common.base.Predicate;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+
+/**
+ * A URIDataPuller has handlings for URI based data
+ */
+public interface URIDataPuller
+{
+ /**
+ * Create a new InputStream based on the URI
+ *
+ * @param uri The URI to open an Input Stream to
+ *
+ * @return A new InputStream which streams the URI in question
+ *
+ * @throws IOException
+ */
+ public InputStream getInputStream(URI uri) throws IOException;
+
+ /**
+ * Returns an abstract "version" for the URI. The exact meaning of the version is left up to the implementation.
+ *
+ * @param uri The URI to check
+ *
+ * @return A "version" as interpreted by the URIDataPuller implementation
+ *
+ * @throws IOException on error
+ */
+ public String getVersion(URI uri) throws IOException;
+
+ /**
+ * Evaluates a Throwable to see if it is recoverable. This is expected to be used in conjunction with the other methods
+ * to determine if anything thrown from the method should be retried.
+ *
+ * @return Predicate function indicating if the Throwable is recoverable
+ */
+ public Predicate shouldRetryPredicate();
+}
diff --git a/api/src/main/java/io/druid/tasklogs/NoopTaskLogs.java b/api/src/main/java/io/druid/tasklogs/NoopTaskLogs.java
new file mode 100644
index 00000000000..d1bfc58bdfb
--- /dev/null
+++ b/api/src/main/java/io/druid/tasklogs/NoopTaskLogs.java
@@ -0,0 +1,46 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.tasklogs;
+
+import com.google.common.base.Optional;
+import com.google.common.io.ByteSource;
+import com.google.common.io.InputSupplier;
+import com.metamx.common.logger.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+
+public class NoopTaskLogs implements TaskLogs
+{
+ private final Logger log = new Logger(TaskLogs.class);
+
+ @Override
+ public Optional streamTaskLog(String taskid, long offset) throws IOException
+ {
+ return Optional.absent();
+ }
+
+ @Override
+ public void pushTaskLog(String taskid, File logFile) throws IOException
+ {
+ log.info("Not pushing logs for task: %s", taskid);
+ }
+}
diff --git a/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java b/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java
new file mode 100644
index 00000000000..e2499fb6c6c
--- /dev/null
+++ b/api/src/main/java/io/druid/tasklogs/TaskLogPusher.java
@@ -0,0 +1,31 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.tasklogs;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Something that knows how to persist local task logs to some form of long-term storage.
+ */
+public interface TaskLogPusher
+{
+ public void pushTaskLog(String taskid, File logFile) throws IOException;
+}
diff --git a/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java b/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java
new file mode 100644
index 00000000000..68d89345ac1
--- /dev/null
+++ b/api/src/main/java/io/druid/tasklogs/TaskLogStreamer.java
@@ -0,0 +1,43 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.tasklogs;
+
+import com.google.common.base.Optional;
+import com.google.common.io.ByteSource;
+import com.google.common.io.InputSupplier;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Something that knows how to stream logs for tasks.
+ */
+public interface TaskLogStreamer
+{
+ /**
+ * Stream log for a task.
+ *
+ * @param offset If zero, stream the entire log. If positive, attempt to read from this position onwards. If
+ * negative, attempt to read this many bytes from the end of the file (like tail -n ).
+ *
+ * @return input supplier for this log, if available from this provider
+ */
+ public Optional streamTaskLog(String taskid, long offset) throws IOException;
+}
diff --git a/api/src/main/java/io/druid/tasklogs/TaskLogs.java b/api/src/main/java/io/druid/tasklogs/TaskLogs.java
new file mode 100644
index 00000000000..f8ba349d7ae
--- /dev/null
+++ b/api/src/main/java/io/druid/tasklogs/TaskLogs.java
@@ -0,0 +1,24 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.tasklogs;
+
+public interface TaskLogs extends TaskLogStreamer, TaskLogPusher
+{
+}
diff --git a/api/src/main/java/io/druid/timeline/DataSegment.java b/api/src/main/java/io/druid/timeline/DataSegment.java
new file mode 100644
index 00000000000..ec9f39bc81f
--- /dev/null
+++ b/api/src/main/java/io/druid/timeline/DataSegment.java
@@ -0,0 +1,418 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.timeline;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Interner;
+import com.google.common.collect.Interners;
+import com.google.common.collect.Iterables;
+import com.metamx.common.Granularity;
+import io.druid.jackson.CommaListJoinDeserializer;
+import io.druid.jackson.CommaListJoinSerializer;
+import io.druid.query.SegmentDescriptor;
+import io.druid.timeline.partition.NoneShardSpec;
+import io.druid.timeline.partition.ShardSpec;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ */
+public class DataSegment implements Comparable
+{
+ public static String delimiter = "_";
+ private final Integer binaryVersion;
+ private static final Interner interner = Interners.newWeakInterner();
+ private static final Function internFun = new Function()
+ {
+ @Override
+ public String apply(String input)
+ {
+ return interner.intern(input);
+ }
+ };
+
+ public static String makeDataSegmentIdentifier(
+ String dataSource,
+ DateTime start,
+ DateTime end,
+ String version,
+ ShardSpec shardSpec
+ )
+ {
+ StringBuilder sb = new StringBuilder();
+
+ sb.append(dataSource).append(delimiter)
+ .append(start).append(delimiter)
+ .append(end).append(delimiter)
+ .append(version);
+
+ if (shardSpec.getPartitionNum() != 0) {
+ sb.append(delimiter).append(shardSpec.getPartitionNum());
+ }
+
+ return sb.toString();
+ }
+
+ private final String dataSource;
+ private final Interval interval;
+ private final String version;
+ private final Map loadSpec;
+ private final List dimensions;
+ private final List metrics;
+ private final ShardSpec shardSpec;
+ private final long size;
+ private final String identifier;
+
+ @JsonCreator
+ public DataSegment(
+ @JsonProperty("dataSource") String dataSource,
+ @JsonProperty("interval") Interval interval,
+ @JsonProperty("version") String version,
+ // use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution
+ @JsonProperty("loadSpec") Map loadSpec,
+ @JsonProperty("dimensions") @JsonDeserialize(using = CommaListJoinDeserializer.class) List dimensions,
+ @JsonProperty("metrics") @JsonDeserialize(using = CommaListJoinDeserializer.class) List metrics,
+ @JsonProperty("shardSpec") ShardSpec shardSpec,
+ @JsonProperty("binaryVersion") Integer binaryVersion,
+ @JsonProperty("size") long size
+ )
+ {
+ final Predicate nonEmpty = new Predicate()
+ {
+ @Override
+ public boolean apply(String input)
+ {
+ return input != null && !input.isEmpty();
+ }
+ };
+
+ // dataSource, dimensions & metrics are stored as canonical string values to decrease memory required for storing large numbers of segments.
+ this.dataSource = interner.intern(dataSource);
+ this.interval = interval;
+ this.loadSpec = loadSpec;
+ this.version = version;
+ this.dimensions = dimensions == null
+ ? ImmutableList.of()
+ : ImmutableList.copyOf(Iterables.transform(Iterables.filter(dimensions, nonEmpty), internFun));
+ this.metrics = metrics == null
+ ? ImmutableList.of()
+ : ImmutableList.copyOf(Iterables.transform(Iterables.filter(metrics, nonEmpty), internFun));
+ this.shardSpec = (shardSpec == null) ? new NoneShardSpec() : shardSpec;
+ this.binaryVersion = binaryVersion;
+ this.size = size;
+
+ this.identifier = makeDataSegmentIdentifier(
+ this.dataSource,
+ this.interval.getStart(),
+ this.interval.getEnd(),
+ this.version,
+ this.shardSpec
+ );
+ }
+
+ /**
+ * Get dataSource
+ *
+ * @return the dataSource
+ */
+ @JsonProperty
+ public String getDataSource()
+ {
+ return dataSource;
+ }
+
+ @JsonProperty
+ public Interval getInterval()
+ {
+ return interval;
+ }
+
+ @JsonProperty
+ public Map getLoadSpec()
+ {
+ return loadSpec;
+ }
+
+ @JsonProperty
+ public String getVersion()
+ {
+ return version;
+ }
+
+ @JsonProperty
+ @JsonSerialize(using = CommaListJoinSerializer.class)
+ public List getDimensions()
+ {
+ return dimensions;
+ }
+
+ @JsonProperty
+ @JsonSerialize(using = CommaListJoinSerializer.class)
+ public List getMetrics()
+ {
+ return metrics;
+ }
+
+ @JsonProperty
+ public ShardSpec getShardSpec()
+ {
+ return shardSpec;
+ }
+
+ @JsonProperty
+ public Integer getBinaryVersion()
+ {
+ return binaryVersion;
+ }
+
+ @JsonProperty
+ public long getSize()
+ {
+ return size;
+ }
+
+ @JsonProperty
+ public String getIdentifier()
+ {
+ return identifier;
+ }
+
+ public SegmentDescriptor toDescriptor()
+ {
+ return new SegmentDescriptor(interval, version, shardSpec.getPartitionNum());
+ }
+
+ public DataSegment withLoadSpec(Map loadSpec)
+ {
+ return builder(this).loadSpec(loadSpec).build();
+ }
+
+ public DataSegment withDimensions(List dimensions)
+ {
+ return builder(this).dimensions(dimensions).build();
+ }
+
+ public DataSegment withSize(long size)
+ {
+ return builder(this).size(size).build();
+ }
+
+ public DataSegment withVersion(String version)
+ {
+ return builder(this).version(version).build();
+ }
+
+ public DataSegment withBinaryVersion(int binaryVersion)
+ {
+ return builder(this).binaryVersion(binaryVersion).build();
+ }
+
+ @Override
+ public int compareTo(DataSegment dataSegment)
+ {
+ return getIdentifier().compareTo(dataSegment.getIdentifier());
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (o instanceof DataSegment) {
+ return getIdentifier().equals(((DataSegment) o).getIdentifier());
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return getIdentifier().hashCode();
+ }
+
+ @Override
+ public String toString()
+ {
+ return "DataSegment{" +
+ "size=" + size +
+ ", shardSpec=" + shardSpec +
+ ", metrics=" + metrics +
+ ", dimensions=" + dimensions +
+ ", version='" + version + '\'' +
+ ", loadSpec=" + loadSpec +
+ ", interval=" + interval +
+ ", dataSource='" + dataSource + '\'' +
+ ", binaryVersion='" + binaryVersion + '\'' +
+ '}';
+ }
+
+ public static Comparator bucketMonthComparator()
+ {
+ return new Comparator()
+ {
+ @Override
+ public int compare(DataSegment lhs, DataSegment rhs)
+ {
+ int retVal;
+
+ DateTime lhsMonth = Granularity.MONTH.truncate(lhs.getInterval().getStart());
+ DateTime rhsMonth = Granularity.MONTH.truncate(rhs.getInterval().getStart());
+
+ retVal = lhsMonth.compareTo(rhsMonth);
+
+ if (retVal != 0) {
+ return retVal;
+ }
+
+ return lhs.compareTo(rhs);
+ }
+ };
+ }
+
+ public static Builder builder()
+ {
+ return new Builder();
+ }
+
+ public static Builder builder(DataSegment segment)
+ {
+ return new Builder(segment);
+ }
+
+ public static class Builder
+ {
+ private String dataSource;
+ private Interval interval;
+ private String version;
+ private Map loadSpec;
+ private List dimensions;
+ private List metrics;
+ private ShardSpec shardSpec;
+ private Integer binaryVersion;
+ private long size;
+
+ public Builder()
+ {
+ this.loadSpec = ImmutableMap.of();
+ this.dimensions = ImmutableList.of();
+ this.metrics = ImmutableList.of();
+ this.shardSpec = new NoneShardSpec();
+ this.size = -1;
+ }
+
+ public Builder(DataSegment segment)
+ {
+ this.dataSource = segment.getDataSource();
+ this.interval = segment.getInterval();
+ this.version = segment.getVersion();
+ this.loadSpec = segment.getLoadSpec();
+ this.dimensions = segment.getDimensions();
+ this.metrics = segment.getMetrics();
+ this.shardSpec = segment.getShardSpec();
+ this.binaryVersion = segment.getBinaryVersion();
+ this.size = segment.getSize();
+ }
+
+ public Builder dataSource(String dataSource)
+ {
+ this.dataSource = dataSource;
+ return this;
+ }
+
+ public Builder interval(Interval interval)
+ {
+ this.interval = interval;
+ return this;
+ }
+
+ public Builder version(String version)
+ {
+ this.version = version;
+ return this;
+ }
+
+ public Builder loadSpec(Map loadSpec)
+ {
+ this.loadSpec = loadSpec;
+ return this;
+ }
+
+ public Builder dimensions(List dimensions)
+ {
+ this.dimensions = dimensions;
+ return this;
+ }
+
+ public Builder metrics(List metrics)
+ {
+ this.metrics = metrics;
+ return this;
+ }
+
+ public Builder shardSpec(ShardSpec shardSpec)
+ {
+ this.shardSpec = shardSpec;
+ return this;
+ }
+
+ public Builder binaryVersion(Integer binaryVersion)
+ {
+ this.binaryVersion = binaryVersion;
+ return this;
+ }
+
+ public Builder size(long size)
+ {
+ this.size = size;
+ return this;
+ }
+
+ public DataSegment build()
+ {
+ // Check stuff that goes into the identifier, at least.
+ Preconditions.checkNotNull(dataSource, "dataSource");
+ Preconditions.checkNotNull(interval, "interval");
+ Preconditions.checkNotNull(version, "version");
+ Preconditions.checkNotNull(shardSpec, "shardSpec");
+
+ return new DataSegment(
+ dataSource,
+ interval,
+ version,
+ loadSpec,
+ dimensions,
+ metrics,
+ shardSpec,
+ binaryVersion,
+ size
+ );
+ }
+ }
+}
diff --git a/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java b/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java
new file mode 100644
index 00000000000..e1b7eb42917
--- /dev/null
+++ b/api/src/main/java/io/druid/timeline/partition/NoneShardSpec.java
@@ -0,0 +1,84 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.timeline.partition;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import com.metamx.common.ISE;
+import io.druid.data.input.InputRow;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+/**
+ */
+public class NoneShardSpec implements ShardSpec
+{
+ @Override
+ public PartitionChunk createChunk(T obj)
+ {
+ return new SingleElementPartitionChunk(obj);
+ }
+
+ @Override
+ public boolean isInChunk(long timestamp, InputRow inputRow)
+ {
+ return true;
+ }
+
+ @Override
+ public int getPartitionNum()
+ {
+ return 0;
+ }
+
+ @Override
+ public ShardSpecLookup getLookup(final List shardSpecs)
+ {
+
+ return new ShardSpecLookup()
+ {
+ @Override
+ public ShardSpec getShardSpec(long timestamp, InputRow row)
+ {
+ return shardSpecs.get(0);
+ }
+ };
+ }
+
+ @Override
+ public boolean equals(Object obj)
+ {
+ return obj instanceof NoneShardSpec;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return 0;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "NoneShardSpec";
+ }
+}
diff --git a/api/src/main/java/io/druid/timeline/partition/PartitionChunk.java b/api/src/main/java/io/druid/timeline/partition/PartitionChunk.java
new file mode 100644
index 00000000000..028350fdf0a
--- /dev/null
+++ b/api/src/main/java/io/druid/timeline/partition/PartitionChunk.java
@@ -0,0 +1,72 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.timeline.partition;
+
+/**
+ * A PartitionChunk represents a chunk of a partitioned(sharded) space. It has knowledge of whether it is
+ * the start of the domain of partitions, the end of the domain, if it abuts another partition and where it stands
+ * inside of a sorted collection of partitions.
+ *
+ * The ordering of PartitionChunks is based entirely upon the partition boundaries defined inside the concrete
+ * PartitionChunk class. That is, the payload (the object returned by getObject()) should *not* be involved in
+ * comparisons between PartitionChunk objects.
+ */
+public interface PartitionChunk extends Comparable>
+{
+ /**
+ * Returns the payload, generally an object that can be used to perform some action against the shard.
+ *
+ * @return the payload
+ */
+ public T getObject();
+
+ /**
+ * Determines if this PartitionChunk abuts another PartitionChunk. A sequence of abutting PartitionChunks should
+ * start with an object where isStart() == true and eventually end with an object where isEnd() == true.
+ *
+ * @param chunk input chunk
+ * @return true if this chunk abuts the input chunk
+ */
+ public boolean abuts(PartitionChunk chunk);
+
+ /**
+ * Returns true if this chunk is the beginning of the partition. Most commonly, that means it represents the range
+ * [-infinity, X) for some concrete X.
+ *
+ * @return true if the chunk is the beginning of the partition
+ */
+ public boolean isStart();
+
+ /**
+ * Returns true if this chunk is the end of the partition. Most commonly, that means it represents the range
+ * [X, infinity] for some concrete X.
+ *
+ * @return true if the chunk is the beginning of the partition
+ */
+ public boolean isEnd();
+
+ /**
+ * Returns the partition chunk number of this PartitionChunk. I.e. if there are 4 partitions in total and this
+ * is the 3rd partition, it will return 2
+ *
+ * @return the sequential numerical id of this partition chunk
+ */
+ public int getChunkNumber();
+}
diff --git a/api/src/main/java/io/druid/timeline/partition/ShardSpec.java b/api/src/main/java/io/druid/timeline/partition/ShardSpec.java
new file mode 100644
index 00000000000..9be668e5705
--- /dev/null
+++ b/api/src/main/java/io/druid/timeline/partition/ShardSpec.java
@@ -0,0 +1,44 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.timeline.partition;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.druid.data.input.InputRow;
+
+import java.util.List;
+
+/**
+ * A Marker interface that exists to combine ShardSpec objects together for Jackson
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+@JsonSubTypes({
+ @JsonSubTypes.Type(name = "none", value = NoneShardSpec.class),
+ })
+public interface ShardSpec
+{
+ public PartitionChunk createChunk(T obj);
+
+ public boolean isInChunk(long timestamp, InputRow inputRow);
+
+ public int getPartitionNum();
+
+ public ShardSpecLookup getLookup(List shardSpecs);
+}
diff --git a/api/src/main/java/io/druid/timeline/partition/ShardSpecLookup.java b/api/src/main/java/io/druid/timeline/partition/ShardSpecLookup.java
new file mode 100644
index 00000000000..5d0e91f25e7
--- /dev/null
+++ b/api/src/main/java/io/druid/timeline/partition/ShardSpecLookup.java
@@ -0,0 +1,27 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.timeline.partition;
+
+import io.druid.data.input.InputRow;
+
+public interface ShardSpecLookup
+{
+ ShardSpec getShardSpec(long timestamp, InputRow row);
+}
diff --git a/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java b/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java
new file mode 100644
index 00000000000..b911af73fbf
--- /dev/null
+++ b/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java
@@ -0,0 +1,109 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.timeline.partition;
+
+/**
+ */
+public class SingleElementPartitionChunk implements PartitionChunk
+{
+ private final T element;
+
+ public SingleElementPartitionChunk
+ (
+ T element
+ )
+ {
+ this.element = element;
+ }
+
+ @Override
+ public T getObject()
+ {
+ return element;
+ }
+
+ @Override
+ public boolean abuts(PartitionChunk tPartitionChunk)
+ {
+ return false;
+ }
+
+ @Override
+ public boolean isStart()
+ {
+ return true;
+ }
+
+ @Override
+ public boolean isEnd()
+ {
+ return true;
+ }
+
+ @Override
+ public int getChunkNumber()
+ {
+ return 0;
+ }
+
+ /**
+ * The ordering of PartitionChunks is determined entirely by the partition boundaries and has nothing to do
+ * with the object. Thus, if there are two SingleElementPartitionChunks, they are equal because they both
+ * represent the full partition space.
+ *
+ * SingleElementPartitionChunks are currently defined as less than every other type of PartitionChunk. There
+ * is no good reason for it, nor is there a bad reason, that's just the way it is. This is subject to change.
+ *
+ * @param chunk
+ * @return
+ */
+ @Override
+ public int compareTo(PartitionChunk chunk)
+ {
+ return chunk instanceof SingleElementPartitionChunk ? 0 : -1;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return element != null ? element.hashCode() : 0;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "SingleElementPartitionChunk{" +
+ "element=" + element +
+ '}';
+ }
+}
diff --git a/api/src/main/java/io/druid/utils/CompressionUtils.java b/api/src/main/java/io/druid/utils/CompressionUtils.java
new file mode 100644
index 00000000000..79ae7047204
--- /dev/null
+++ b/api/src/main/java/io/druid/utils/CompressionUtils.java
@@ -0,0 +1,81 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.utils;
+
+import com.metamx.common.logger.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ */
+public class CompressionUtils
+{
+ private static final Logger log = new Logger(CompressionUtils.class);
+
+
+ @Deprecated // Use com.metamx.common.CompressionUtils.zip
+ public static long zip(File directory, File outputZipFile) throws IOException
+ {
+ return com.metamx.common.CompressionUtils.zip(directory, outputZipFile);
+ }
+
+
+ @Deprecated // Use com.metamx.common.CompressionUtils.zip
+ public static long zip(File directory, OutputStream out) throws IOException
+ {
+ return com.metamx.common.CompressionUtils.zip(directory, out);
+ }
+
+ @Deprecated // Use com.metamx.common.CompressionUtils.unzip
+ public static void unzip(File pulledFile, File outDir) throws IOException
+ {
+ com.metamx.common.CompressionUtils.unzip(pulledFile, outDir);
+ }
+
+ @Deprecated // Use com.metamx.common.CompressionUtils.unzip
+ public static void unzip(InputStream in, File outDir) throws IOException
+ {
+ com.metamx.common.CompressionUtils.unzip(in, outDir);
+ }
+
+ /**
+ * Uncompress using a gzip uncompress algorithm from the `pulledFile` to the `outDir`.
+ * Unlike `com.metamx.common.CompressionUtils.gunzip`, this function takes an output *DIRECTORY* and tries to guess the file name.
+ * It is recommended that the caller use `com.metamx.common.CompressionUtils.gunzip` and specify the output file themselves to ensure names are as expected
+ *
+ * @param pulledFile The source file
+ * @param outDir The destination directory to put the resulting file
+ *
+ * @throws IOException on propogated IO exception, IAE if it cannot determine the proper new name for `pulledFile`
+ */
+ @Deprecated // See description for alternative
+ public static void gunzip(File pulledFile, File outDir) throws IOException
+ {
+ final File outFile = new File(outDir, com.metamx.common.CompressionUtils.getGzBaseName(pulledFile.getName()));
+ com.metamx.common.CompressionUtils.gunzip(pulledFile, outFile);
+ if (!pulledFile.delete()) {
+ log.error("Could not delete tmpFile[%s].", pulledFile);
+ }
+ }
+
+}
diff --git a/api/src/main/java/io/druid/utils/Runnables.java b/api/src/main/java/io/druid/utils/Runnables.java
new file mode 100644
index 00000000000..a7491febb1e
--- /dev/null
+++ b/api/src/main/java/io/druid/utils/Runnables.java
@@ -0,0 +1,31 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid.utils;
+
+/**
+ */
+public class Runnables
+{
+ public static Runnable getNoopRunnable(){
+ return new Runnable(){
+ public void run(){}
+ };
+ }
+}
diff --git a/api/src/test/java/io/druid/TestObjectMapper.java b/api/src/test/java/io/druid/TestObjectMapper.java
new file mode 100644
index 00000000000..6846c17020d
--- /dev/null
+++ b/api/src/test/java/io/druid/TestObjectMapper.java
@@ -0,0 +1,70 @@
+/*
+* Licensed to Metamarkets Group Inc. (Metamarkets) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. Metamarkets licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+
+package io.druid;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
+import org.joda.time.Interval;
+
+import java.io.IOException;
+
+/**
+ */
+public class TestObjectMapper extends ObjectMapper
+{
+ public TestObjectMapper()
+ {
+ configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+ configure(MapperFeature.AUTO_DETECT_GETTERS, false);
+ configure(MapperFeature.AUTO_DETECT_FIELDS, false);
+ configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false);
+ configure(MapperFeature.AUTO_DETECT_SETTERS, false);
+ configure(SerializationFeature.INDENT_OUTPUT, false);
+ registerModule(new TestModule());
+ }
+
+ public static class TestModule extends SimpleModule
+ {
+ TestModule()
+ {
+ addSerializer(Interval.class, ToStringSerializer.instance);
+ addDeserializer(
+ Interval.class, new StdDeserializer