Merge pull request #714 from metamx/cleanup-javadocs

Cleanup javadocs
This commit is contained in:
fjy 2014-09-04 12:28:10 -06:00
commit f473e62f1f
148 changed files with 300 additions and 359 deletions

View File

@ -34,13 +34,13 @@ import java.util.PriorityQueue;
/**
* An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming
* that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators.
* <p/>
*
* Normally a merge operation could operate with the actual input iterators in any order as long as the actual values
* in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators
* be provided in the order of the first element of each iterator.
* <p/>
*
* If this doesn't make sense, check out OrderedMergeIteratorTest.testScrewsUpOnOutOfOrderBeginningOfList()
* <p/>
*
* It places this extra restriction on the input data in order to implement an optimization that allows it to
* remain as lazy as possible in the face of a common case where the iterators are just appended one after the other.
*/

View File

@ -22,7 +22,6 @@ package io.druid.collections;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Ordering;
import com.google.common.io.Closeables;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.Sequence;
@ -38,13 +37,13 @@ import java.util.PriorityQueue;
/**
* An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming
* that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators.
* <p/>
*
* Normally a merge operation could operate with the actual input iterators in any order as long as the actual values
* in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators
* be provided in the order of the first element of each iterator.
* <p/>
*
* If this doesn't make sense, check out OrderedMergeSequenceTest.testScrewsUpOnOutOfOrderBeginningOfList()
* <p/>
*
* It places this extra restriction on the input data in order to implement an optimization that allows it to
* remain as lazy as possible in the face of a common case where the iterators are just appended one after the other.
*/

View File

@ -20,7 +20,6 @@
package io.druid.common.utils;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.OutputSupplier;
import io.druid.collections.IntList;
@ -263,4 +262,4 @@ public class SerializerUtils
return retVal;
}
}
}

View File

@ -22,8 +22,6 @@ package io.druid.db;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import javax.validation.constraints.NotNull;
/**
*/
public class DbTablesConfig
@ -122,4 +120,4 @@ public class DbTablesConfig
{
return taskLockTable;
}
}
}

View File

@ -42,17 +42,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
/**
* VersionedIntervalTimeline is a data structure that manages objects on a specific timeline.
* <p/>
*
* It associates a jodatime Interval and a generically-typed version with the object that is being stored.
* <p/>
*
* In the event of overlapping timeline entries, timeline intervals may be chunked. The underlying data associated
* with a timeline entry remains unchanged when chunking occurs.
* <p/>
*
* After loading objects via the add() method, the lookup(Interval) method can be used to get the list of the most
* recent objects (according to the version) that match the given interval. The intent is that objects represent
* a certain time period and when you do a lookup(), you are asking for all of the objects that you need to look
* at in order to get a correct answer about that time period.
* <p/>
*
* The findOvershadowed() method returns a list of objects that will never be returned by a call to lookup() because
* they are overshadowed by some other object. This can be used in conjunction with the add() and remove() methods
* to achieve "atomic" updates. First add new items, then check if those items caused anything to be overshadowed, if

View File

@ -28,8 +28,6 @@ import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.SequenceTestHelper;
import com.metamx.common.guava.Sequences;
import com.metamx.common.guava.TestSequence;
import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.YieldingAccumulator;
import junit.framework.Assert;
import org.junit.Test;

View File

@ -45,6 +45,7 @@ import static java.lang.Thread.sleep;
* The generated tuples can be thought of as asynchronously
* produced triples (timestamp, outColumn, target) where the timestamp varies depending on
* speed of processing.
*
* <p>
* InputRows are produced as fast as requested, so this can be used to determine the
* upper rate of ingest if sleepUsec is set to 0; nTokens specifies how many associated
@ -55,8 +56,8 @@ import static java.lang.Thread.sleep;
* the moment an event is delivered.)
* Values are offset by adding the modulus of the token number to the random number
* so that token values have distinct, non-overlapping ranges.
* <p/>
* </p>
*
* Example spec file:
* <pre>
* [{
@ -82,7 +83,7 @@ import static java.lang.Thread.sleep;
* "basePersistDirectory" : "/tmp/realtime/basePersist" }
* }]
* </pre>
* <p/>
*
* Example query using POST to /druid/v2/ (where UTC date and time MUST include the current hour):
* <pre>
* {

View File

@ -58,18 +58,11 @@ import static java.lang.Thread.sleep;
* with timestamps along with ??.
* The generated tuples have the form (timestamp, ????)
* where the timestamp is from the twitter event.
* <p>
* <p/>
* </p>
*
* Example spec file:
* <pre>
* </pre>
* <p/>
*
* Example query using POST to /druid/v2/?w (where w is an arbitrary parameter and the date and time
* is UTC):
* <pre>
* </pre>
*
*
* Notes on twitter.com HTTP (REST) API: v1.0 will be disabled around 2013-03 so v1.1 should be used;
* twitter4j 3.0 (not yet released) will support the v1.1 api.

View File

@ -21,9 +21,6 @@ package io.druid.examples.web;
import org.junit.Test;
import java.io.IOException;
import java.net.MalformedURLException;
public class WebJsonSupplierTest
{
@Test(expected = IllegalStateException.class)

View File

@ -19,9 +19,7 @@
package io.druid.storage.hdfs;
import com.google.common.io.Closeables;
import com.google.inject.Inject;
import com.metamx.common.guava.CloseQuietly;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;

View File

@ -20,13 +20,10 @@
package io.druid.storage.hdfs;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.OutputSupplier;
import com.google.inject.Inject;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;

View File

@ -253,9 +253,7 @@ public class ApproximateHistogram
}
/**
* Returns a string representation of the actual bin counts
*
* @return
* @return a string representation of the actual bin counts
*/
protected String getBinsString()
{
@ -385,7 +383,7 @@ public class ApproximateHistogram
/**
* Merges the bin in the given position with the next bin
*
* @param index index of the bin to merge, index must satisfy 0 <= index < binCount - 1
* @param index index of the bin to merge, index must satisfy 0 &lt;= index &lt; binCount - 1
*/
protected void merge(final int index)
{
@ -506,9 +504,9 @@ public class ApproximateHistogram
/**
* Copies histogram h into the current histogram.
*
* @param h
* @param h ApproximateHistogram to copy
*
* @return
* @return this histogram
*/
public ApproximateHistogram copy(ApproximateHistogram h)
{
@ -815,20 +813,20 @@ public class ApproximateHistogram
/**
* mergeBins performs the given number of bin merge operations on the given histogram
* <p/>
*
* It repeatedly merges the two closest bins until it has performed the requested number of merge operations.
* Merges are done in-place and unused bins have unknown state
* <p/>
*
* next / prev maintains a doubly-linked list of valid bin indices into the mergedBins array.
* <p/>
*
* Fast operation is achieved by building a min-heap of the deltas as opposed to repeatedly
* scanning the array of deltas to find the minimum. A reverse index into the heap is maintained
* to allow deleting and updating of specific deltas.
* <p/>
*
* next and prev arrays are used to maintain indices to the previous / next valid bin from a given bin index
* <p/>
*
* Its effect is equivalent to running the following code:
* <p/>
*
* <pre>
* ApproximateHistogram merged = new ApproximateHistogram(mergedBinCount, mergedPositions, mergedBins);
*
@ -1104,7 +1102,7 @@ public class ApproximateHistogram
/**
* Returns a byte-array representation of this ApproximateHistogram object
*
* @return
* @return byte array representation
*/
@JsonValue
public byte[] toBytes()
@ -1181,7 +1179,7 @@ public class ApproximateHistogram
/**
* Writes the representation of this ApproximateHistogram object to the given byte-buffer
*
* @param buf
* @param buf ByteBuffer to write the ApproximateHistogram to
*/
public void toBytes(ByteBuffer buf)
{
@ -1196,10 +1194,10 @@ public class ApproximateHistogram
/**
* Writes the dense representation of this ApproximateHistogram object to the given byte-buffer
* <p/>
*
* Requires 16 + 12 * size bytes of storage
*
* @param buf
* @param buf ByteBuffer to write the ApproximateHistogram to
*/
public void toBytesDense(ByteBuffer buf)
{
@ -1217,10 +1215,10 @@ public class ApproximateHistogram
/**
* Writes the sparse representation of this ApproximateHistogram object to the given byte-buffer
* <p/>
*
* Requires 16 + 12 * binCount bytes of storage
*
* @param buf ByteBuffer to write object to
* @param buf ByteBuffer to write the ApproximateHistogram to
*/
public void toBytesSparse(ByteBuffer buf)
{
@ -1239,10 +1237,10 @@ public class ApproximateHistogram
/**
* Returns a compact byte-buffer representation of this ApproximateHistogram object
* storing actual values as opposed to histogram bins
* <p/>
* Requires 3 + 4 * count bytes of storage with count <= 127
*
* Requires 3 + 4 * count bytes of storage with count &lt;= 127
*
* @param buf
* @param buf ByteBuffer to write the ApproximateHistogram to
*/
public void toBytesCompact(ByteBuffer buf)
{
@ -1285,9 +1283,9 @@ public class ApproximateHistogram
/**
* Constructs an Approximate Histogram object from the given byte-array representation
*
* @param bytes
* @param bytes byte array to construct an ApproximateHistogram from
*
* @return
* @return ApproximateHistogram constructed from the given byte array
*/
public static ApproximateHistogram fromBytes(byte[] bytes)
{
@ -1298,9 +1296,9 @@ public class ApproximateHistogram
/**
* Constructs an ApproximateHistogram object from the given dense byte-buffer representation
*
* @param buf
* @param buf ByteBuffer to construct an ApproximateHistogram from
*
* @return
* @return ApproximateHistogram constructed from the given ByteBuffer
*/
public static ApproximateHistogram fromBytesDense(ByteBuffer buf)
{
@ -1324,9 +1322,9 @@ public class ApproximateHistogram
/**
* Constructs an ApproximateHistogram object from the given dense byte-buffer representation
*
* @param buf
* @param buf ByteBuffer to construct an ApproximateHistogram from
*
* @return
* @return ApproximateHistogram constructed from the given ByteBuffer
*/
public static ApproximateHistogram fromBytesSparse(ByteBuffer buf)
{
@ -1352,9 +1350,9 @@ public class ApproximateHistogram
/**
* Constructs an ApproximateHistogram object from the given compact byte-buffer representation
*
* @param buf
* @param buf ByteBuffer to construct an ApproximateHistogram from
*
* @return
* @return ApproximateHistogram constructed from the given ByteBuffer
*/
public static ApproximateHistogram fromBytesCompact(ByteBuffer buf)
{
@ -1428,9 +1426,9 @@ public class ApproximateHistogram
/**
* Constructs an ApproximateHistogram object from the given byte-buffer representation
*
* @param buf
* @param buf ByteBuffer to construct an ApproximateHistogram from
*
* @return
* @return ApproximateHistogram constructed from the given ByteBuffer
*/
public static ApproximateHistogram fromBytes(ByteBuffer buf)
{
@ -1454,7 +1452,7 @@ public class ApproximateHistogram
/**
* Returns the approximate number of items less than or equal to b in the histogram
*
* @param b
* @param b the cutoff
*
* @return the approximate number of items less than or equal to b
*/
@ -1521,7 +1519,7 @@ public class ApproximateHistogram
* probabilities = [.5f] returns [median]
* probabilities = [.25f, .5f, .75f] returns the quartiles, [25%ile, median, 75%ile]
*
* @param probabilities
* @param probabilities array of probabilities
*
* @return an array of length probabilities.length representing the the approximate sample quantiles
* corresponding to the given probabilities
@ -1584,9 +1582,9 @@ public class ApproximateHistogram
/**
* Computes a visual representation of the approximate histogram with bins laid out according to the given breaks
*
* @param breaks
* @param breaks breaks defining the histogram bins
*
* @return
* @return visual representation of the histogram
*/
public Histogram toHistogram(final float[] breaks)
{
@ -1607,7 +1605,7 @@ public class ApproximateHistogram
*
* @param size number of equal-sized bins to divide the histogram into
*
* @return
* @return visual representation of the histogram
*/
public Histogram toHistogram(int size)
{
@ -1629,7 +1627,7 @@ public class ApproximateHistogram
* @param bucketSize the size of each bucket
* @param offset the location of one breakpoint
*
* @return
* @return visual representation of the histogram
*/
public Histogram toHistogram(final float bucketSize, final float offset)
{

View File

@ -81,13 +81,13 @@ import java.util.Map;
/**
* Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so,
* choosing the best dimension that satisfies the criteria:
* <p/>
*
* <ul>
* <li>Must have exactly one value per row.</li>
* <li>Must not generate oversized partitions. A dimension with N rows having the same value will necessarily
* put all those rows in the same partition, and that partition may be much larger than the target size.</li>
* </ul>
* <p/>
*
* "Best" means a very high cardinality dimension, or, if none exist, the dimension that minimizes variation of
* segment size relative to the target.
*/

View File

@ -23,12 +23,12 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import io.druid.db.DbConnectorConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.RandomPartitionsSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.indexer.updater.DbUpdaterJobSpec;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;

View File

@ -20,7 +20,6 @@
package io.druid.indexer.partitions;
import com.google.common.base.Throwables;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.HadoopDruidIndexerConfigTest;
import org.junit.Assert;
import org.junit.Test;

View File

@ -20,7 +20,6 @@
package io.druid.indexer.partitions;
import com.google.common.base.Throwables;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.HadoopDruidIndexerConfigTest;
import org.junit.Assert;
import org.junit.Test;

View File

@ -28,7 +28,7 @@ import com.google.common.base.Preconditions;
/**
* Represents the status of a task from the perspective of the coordinator. The task may be ongoing
* ({@link #isComplete()} false) or it may be complete ({@link #isComplete()} true).
* <p/>
*
* TaskStatus objects are immutable.
*/
public class TaskStatus
@ -101,6 +101,8 @@ public class TaskStatus
/**
* Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable,
* isSuccess, or isFailure will be true at any one time.
*
* @return whether the task is runnable.
*/
@JsonIgnore
public boolean isRunnable()
@ -110,6 +112,8 @@ public class TaskStatus
/**
* Inverse of {@link #isRunnable}.
*
* @return whether the task is complete.
*/
@JsonIgnore
public boolean isComplete()
@ -120,6 +124,8 @@ public class TaskStatus
/**
* Returned by tasks when they spawn subtasks. Exactly one of isRunnable, isSuccess, or isFailure will
* be true at any one time.
*
* @return whether the task succeeded.
*/
@JsonIgnore
public boolean isSuccess()
@ -130,6 +136,8 @@ public class TaskStatus
/**
* Returned by tasks when they complete unsuccessfully. Exactly one of isRunnable, isSuccess, or
* isFailure will be true at any one time.
*
* @return whether the task failed
*/
@JsonIgnore
public boolean isFailure()

View File

@ -29,7 +29,6 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.Task;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover;

View File

@ -124,7 +124,10 @@ public abstract class AbstractTask implements Task
}
/**
* Start helper methods *
* Start helper methods
*
* @param objects objects to join
* @return string of joined objects
*/
public static String joinId(Object... objects)
{

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.metamx.common.logger.Logger;
@ -31,7 +30,6 @@ import io.druid.granularity.QueryGranularity;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexableAdapter;

View File

@ -31,6 +31,8 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.metamx.common.logger.Logger;
import io.druid.common.utils.JodaUtils;
import io.druid.guice.ExtensionsConfig;
import io.druid.guice.GuiceInjectors;
import io.druid.indexer.HadoopDruidDetermineConfigurationJob;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.HadoopDruidIndexerJob;
@ -42,9 +44,7 @@ import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockTryAcquireAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization;
import io.druid.guice.ExtensionsConfig;
import io.druid.timeline.DataSegment;
import io.tesla.aether.internal.DefaultTeslaAether;
import org.joda.time.DateTime;
@ -85,7 +85,7 @@ public class HadoopIndexTask extends AbstractTask
/**
* @param spec is used by the HadoopDruidIndexerJob to set up the appropriate parameters
* for creating Druid index segments. It may be modified.
* <p/>
*
* Here, we will ensure that the DbConnectorConfig field of the spec is set to null, such that the
* job does not push a list of published segments the database. Instead, we will use the method
* IndexGeneratorJob.getPublishedSegments() to simply return a list of the published

View File

@ -41,7 +41,6 @@ import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.segment.IndexIO;

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import io.druid.indexing.common.TaskLock;

View File

@ -19,7 +19,6 @@
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
@ -44,7 +43,6 @@ import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.QueryToolChest;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig;
import io.druid.segment.indexing.RealtimeTuningConfig;

View File

@ -59,23 +59,27 @@ public interface Task
{
/**
* Returns ID of this task. Must be unique across all tasks ever created.
* @return task ID
*/
public String getId();
/**
* Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks,
* a common convention is to set group ID equal to task ID.
* @return task group ID
*/
public String getGroupId();
/**
* Returns a {@link io.druid.indexing.common.task.TaskResource} for this task. Task resources define specific
* worker requirements a task may require.
* @return {@link io.druid.indexing.common.task.TaskResource} for this task
*/
public TaskResource getTaskResource();
/**
* Returns a descriptive label for this task type. Used for metrics emission and logging.
* @return task type label
*/
public String getType();
@ -95,6 +99,8 @@ public interface Task
/**
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
* should return null.
* @param <T> query result type
* @return query runners for this task
*/
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
@ -128,7 +134,7 @@ public interface Task
*
* @return Some kind of finished status (isRunnable must be false).
*
* @throws Exception
* @throws Exception if this task failed
*/
public TaskStatus run(TaskToolbox toolbox) throws Exception;
}

View File

@ -43,6 +43,8 @@ public class TaskResource
* Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same
* worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the
* task ID.
*
* @return task availability group
*/
@JsonProperty
public String getAvailabilityGroup()
@ -52,7 +54,7 @@ public class TaskResource
/**
* Returns the number of worker slots this task will take.
* @return the number of worker slots this task will take
*/
@JsonProperty
public int getRequiredCapacity()

View File

@ -39,12 +39,10 @@ import org.joda.time.Interval;
import org.skife.jdbi.v2.FoldController;
import org.skife.jdbi.v2.Folder3;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.ResultIterator;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.TransactionCallback;
import org.skife.jdbi.v2.TransactionStatus;
import org.skife.jdbi.v2.exceptions.CallbackFailedException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import java.io.IOException;
@ -140,6 +138,8 @@ public class IndexerDBCoordinator
*
* @param segments set of segments to add
* @return set of segments actually added
*
* @throws java.io.IOException if a database error occurs
*/
public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException
{

View File

@ -88,13 +88,13 @@ import java.util.concurrent.TimeUnit;
* creating ephemeral nodes in ZK that workers must remove. Workers announce the statuses of the tasks they are running.
* Once a task completes, it is up to the RTR to remove the task status and run any necessary cleanup.
* The RemoteTaskRunner is event driven and updates state according to ephemeral node changes in ZK.
* <p/>
*
* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will
* fail. The RemoteTaskRunner depends on another component to create additional worker resources.
* For example, {@link io.druid.indexing.overlord.scaling.ResourceManagementScheduler} can take care of these duties.
* <p/>
*
* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will fail any tasks associated with the worker.
* <p/>
*
* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages.
*/
public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer

View File

@ -161,6 +161,12 @@ public class TaskLockbox
/**
* Acquires a lock on behalf of a task. Blocks until the lock is acquired. Throws an exception if the lock
* cannot be acquired.
*
* @param task task to acquire lock for
* @param interval interval to lock
* @return acquired TaskLock
*
* @throws java.lang.InterruptedException if the lock cannot be acquired
*/
public TaskLock lock(final Task task, final Interval interval) throws InterruptedException
{
@ -296,6 +302,7 @@ public class TaskLockbox
* Return the currently-active locks for some task.
*
* @param task task for which to locate locks
* @return currently-active locks for the given task
*/
public List<TaskLock> findLocksForTask(final Task task)
{

View File

@ -33,7 +33,6 @@ import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
@ -44,8 +43,6 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import java.util.List;
import java.util.Map;
@ -59,11 +56,11 @@ import java.util.concurrent.locks.ReentrantLock;
/**
* Interface between task producers and the task runner.
* <p/>
*
* This object accepts tasks from producers using {@link #add} and manages delivery of these tasks to a
* {@link TaskRunner}. Tasks will run in a mostly-FIFO order, with deviations when the natural next task is not ready
* in time (based on its {@link Task#isReady} method).
* <p/>
*
* For persistence, we save all new tasks and task status changes using a {@link TaskStorage} object.
*/
public class TaskQueue
@ -295,6 +292,7 @@ public class TaskQueue
* @param task task to add
*
* @return true
* @throws TaskExistsException if the task already exists
*/
public boolean add(final Task task) throws TaskExistsException
{

View File

@ -42,6 +42,8 @@ public interface TaskRunner
/**
* Inform the task runner it can clean up any resources associated with a task. This implies shutdown of any
* currently-running tasks.
*
* @param taskid task ID to clean up resources for
*/
public void shutdown(String taskid);

View File

@ -32,24 +32,33 @@ public interface TaskStorage
/**
* Adds a task to the storage facility with a particular status.
*
* @param task task to add
* @param status task status
* @throws io.druid.indexing.overlord.TaskExistsException if the task ID already exists
*/
public void insert(Task task, TaskStatus status) throws TaskExistsException;
/**
* Persists task status in the storage facility. This method should throw an exception if the task status lifecycle
* is not respected (absent -> RUNNING -> SUCCESS/FAILURE).
* is not respected (absent -&gt; RUNNING -&gt; SUCCESS/FAILURE).
*
* @param status task status
*/
public void setStatus(TaskStatus status);
/**
* Persists lock state in the storage facility.
* @param taskid task ID
* @param taskLock lock state
*/
public void addLock(String taskid, TaskLock taskLock);
/**
* Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but
* this method can help reclaim wasted space.
*
* @param taskid task ID
* @param taskLock lock state
*/
public void removeLock(String taskid, TaskLock taskLock);
@ -58,28 +67,44 @@ public interface TaskStorage
* absentee Optional.
*
* NOTE: This method really feels like it should be combined with {@link #getStatus}. Expect that in the future.
*
* @param taskid task ID
* @return optional task
*/
public Optional<Task> getTask(String taskid);
/**
* Returns task status as stored in the storage facility. If the task ID does not exist, this will return
* an absentee Optional.
*
* @param taskid task ID
* @return task status
*/
public Optional<TaskStatus> getStatus(String taskid);
/**
* Add an action taken by a task to the audit log.
*
* @param task task to record action for
* @param taskAction task action to record
*
* @param <T> task action return type
*/
public <T> void addAuditLog(Task task, TaskAction<T> taskAction);
/**
* Returns all actions taken by a task.
*
* @param taskid task ID
* @return list of task actions
*/
public List<TaskAction> getAuditLogs(String taskid);
/**
* Returns a list of currently running or pending tasks as stored in the storage facility. No particular order
* is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
*
* @return list of active tasks
*/
public List<Task> getActiveTasks();
@ -87,11 +112,16 @@ public interface TaskStorage
* Returns a list of recently finished task statuses as stored in the storage facility. No particular order
* is guaranteed, but implementations are encouraged to return tasks in descending order of creation. No particular
* standard of "recent" is guaranteed, and in fact, this method is permitted to simply return nothing.
*
* @return list of recently finished tasks
*/
public List<TaskStatus> getRecentlyFinishedTaskStatuses();
/**
* Returns a list of locks for a particular task.
*
* @param taskid task ID
* @return list of TaskLocks for the given task
*/
public List<TaskLock> getLocks(String taskid);
}

View File

@ -19,9 +19,7 @@
package io.druid.indexing.overlord;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import io.druid.indexing.common.TaskStatus;
@ -30,7 +28,6 @@ import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.task.Task;
import io.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Set;
@ -73,6 +70,9 @@ public class TaskStorageQueryAdapter
* This method is useful when you want to figure out all of the things a single task spawned. It does pose issues
* with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your
* own risk and know that at some point, we might adjust this to actually enforce some sort of limits.
*
* @param taskid task ID
* @return set of segments created by the specified task
*/
public Set<DataSegment> getInsertedSegments(final String taskid)
{

View File

@ -39,7 +39,6 @@ import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.task.Task;
import io.druid.query.NoopQueryRunner;
import io.druid.query.Query;
import io.druid.query.TableDataSource;
import io.druid.query.QueryRunner;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.SegmentDescriptor;

View File

@ -35,6 +35,9 @@ public interface EC2UserData<T extends EC2UserData>
/**
* Return a copy of this instance with a different worker version. If no changes are needed (possibly because the
* user data does not depend on the worker version) then it is OK to return "this".
*
* @param version worker version
* @return instance with the specified version
*/
public EC2UserData<T> withVersion(String version);

View File

@ -41,7 +41,7 @@ import java.util.concurrent.ExecutorService;
/**
* The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be
* created that waits for new tasks. Tasks are executed as soon as they are seen.
* <p/>
*
* The monitor implements {@link io.druid.query.QuerySegmentWalker} so tasks can offer up queryable data. This is useful for
* realtime index tasks.
*/

View File

@ -19,7 +19,6 @@
package io.druid.indexing.common;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.task.Task;
/**

View File

@ -19,7 +19,6 @@
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;

View File

@ -43,14 +43,12 @@ import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.granularity.QueryGranularity;
import io.druid.indexing.common.TestUtils;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.actions.LocalTaskActionClientFactory;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.SegmentInsertAction;
@ -67,6 +65,7 @@ import io.druid.indexing.overlord.config.TaskQueueConfig;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover;

View File

@ -19,7 +19,6 @@
package io.druid.indexing.overlord.scaling;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;

View File

@ -23,8 +23,6 @@ import com.google.inject.Binder;
import com.google.inject.Module;
import com.google.inject.Provides;
import com.metamx.common.config.Config;
import io.druid.guice.JsonConfigurator;
import io.druid.guice.LazySingleton;
import org.skife.config.ConfigurationObjectFactory;
import javax.validation.Validation;

View File

@ -24,13 +24,6 @@ import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.google.inject.util.Modules;
import io.druid.guice.ConfigModule;
import io.druid.guice.DruidGuiceExtensions;
import io.druid.guice.DruidSecondaryModule;
import io.druid.guice.ExtensionsConfig;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.PropertiesModule;
import io.druid.jackson.JacksonModule;
import java.util.List;

View File

@ -20,12 +20,8 @@
package io.druid.query;
import com.google.common.collect.Lists;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.Yielders;
import com.metamx.common.guava.YieldingAccumulator;
import org.joda.time.DateTime;
import java.util.Arrays;

View File

@ -47,14 +47,14 @@ import java.util.concurrent.TimeoutException;
/**
* A QueryRunner that combines a list of other QueryRunners and executes them in parallel on an executor.
* <p/>
*
* When using this, it is important to make sure that the list of QueryRunners provided is fully flattened.
* If, for example, you were to pass a list of a Chained QueryRunner (A) and a non-chained QueryRunner (B). Imagine
* A has 2 QueryRunner chained together (Aa and Ab), the fact that the Queryables are run in parallel on an
* executor would mean that the Queryables are actually processed in the order
* <p/>
* A -> B -> Aa -> Ab
* <p/>
*
* <pre>A -&gt; B -&gt; Aa -&gt; Ab</pre>
*
* That is, the two sub queryables for A would run *after* B is run, effectively meaning that the results for B
* must be fully cached in memory before the results for Aa and Ab are computed.
*/

View File

@ -59,9 +59,9 @@ public class Druids
/**
* A Builder for AndDimFilter.
* <p/>
*
* Required: fields() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* AndDimFilter andDimFilter = Druids.newAndDimFilterBuilder()
@ -105,9 +105,9 @@ public class Druids
/**
* A Builder for OrDimFilter.
* <p/>
*
* Required: fields() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* OrDimFilter orDimFilter = Druids.newOrDimFilterBuilder()
@ -160,9 +160,9 @@ public class Druids
/**
* A Builder for NotDimFilter.
* <p/>
*
* Required: field() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* NotDimFilter notDimFilter = Druids.newNotDimFilterBuilder()
@ -206,9 +206,9 @@ public class Druids
/**
* A Builder for SelectorDimFilter.
* <p/>
*
* Required: dimension() and value() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* Selector selDimFilter = Druids.newSelectorDimFilterBuilder()
@ -285,10 +285,10 @@ public class Druids
/**
* A Builder for TimeseriesQuery.
* <p/>
*
* Required: dataSource(), intervals(), and aggregators() must be called before build()
* Optional: filters(), granularity(), postAggregators(), and context() can be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
@ -483,11 +483,11 @@ public class Druids
/**
* A Builder for SearchQuery.
* <p/>
*
* Required: dataSource(), intervals(), dimensions() and query() must be called before build()
* <p/>
*
* Optional: filters(), granularity(), and context() can be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* SearchQuery query = Druids.newSearchQueryBuilder()
@ -678,9 +678,9 @@ public class Druids
/**
* A Builder for TimeBoundaryQuery.
* <p/>
*
* Required: dataSource() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* TimeBoundaryQuery query = new MaxTimeQueryBuilder()
@ -774,12 +774,12 @@ public class Druids
/**
* A Builder for Result.
* <p/>
*
* Required: timestamp() and value() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* Result<T> result = Druids.newResultBuilder()
* Result&lt;T&gt; result = Druids.newResultBuilder()
* .timestamp(egDateTime)
* .value(egValue)
* .build();
@ -840,9 +840,9 @@ public class Druids
/**
* A Builder for SegmentMetadataQuery.
* <p/>
*
* Required: dataSource(), intervals() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* SegmentMetadataQuery query = new SegmentMetadataQueryBuilder()
@ -948,9 +948,9 @@ public class Druids
/**
* A Builder for SelectQuery.
* <p/>
*
* Required: dataSource(), intervals() must be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* SelectQuery query = new SelectQueryBuilder()

View File

@ -28,8 +28,6 @@ import com.metamx.common.guava.Sequences;
import io.druid.query.aggregation.MetricManipulationFn;
import io.druid.query.aggregation.MetricManipulatorFns;
import javax.annotation.Nullable;
/**
*/
public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>

View File

@ -21,7 +21,6 @@
package io.druid.query;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
@JsonTypeName("table")

View File

@ -22,7 +22,6 @@ package io.druid.query;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicates;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;

View File

@ -29,6 +29,8 @@ public interface QuerySegmentWalker
* Gets the Queryable for a given interval, the Queryable returned can be any version(s) or partitionNumber(s)
* such that it represents the interval.
*
* @param <T> query result type
* @param query the query to find a Queryable for
* @param intervals the intervals to find a Queryable for
* @return a Queryable object that represents the interval
*/
@ -36,8 +38,10 @@ public interface QuerySegmentWalker
/**
* Gets the Queryable for a given list of SegmentSpecs.
* exist.
*
* @param <T> the query result type
* @param query the query to return a Queryable for
* @param specs the list of SegmentSpecs to find a Queryable for
* @return the Queryable object with the given SegmentSpecs
*/
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs);

View File

@ -40,9 +40,9 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
/**
* This method doesn't belong here, but it's here for now just to make it work.
*
* @param seqOfSequences
* @param seqOfSequences sequence of sequences to be merged
*
* @return
* @return the sequence of merged results
*/
public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);

View File

@ -23,7 +23,6 @@ package io.druid.query;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Lists;
import java.util.Arrays;
import java.util.List;

View File

@ -134,7 +134,7 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
/**
* Map time t into the last `period` ending within `dataInterval`
*
* @param t
* @param t the current time to be mapped into `dataInterval`
* @return the offset between the mapped time and time t
*/
protected long computeOffset(final long t)

View File

@ -26,8 +26,6 @@ import com.google.common.collect.Lists;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import java.util.ArrayList;
public class UnionQueryRunner<T> implements QueryRunner<T>
{
private final QueryRunner<T> baseRunner;

View File

@ -26,9 +26,9 @@ import java.util.List;
/**
* Processing related interface
* <p/>
*
* An AggregatorFactory is an object that knows how to generate an Aggregator using a ColumnSelectorFactory.
* <p/>
*
* This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects
* without making any assumptions about how they are pulling values out of the base data. That is, the data is
* provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how

View File

@ -20,7 +20,6 @@
package io.druid.query.aggregation;
import com.google.common.collect.Lists;
import com.metamx.common.ISE;
import com.metamx.common.Pair;
import java.util.HashSet;
@ -37,6 +36,8 @@ public class AggregatorUtil
* such that all the dependencies of any given aggregator should occur before that aggregator.
* See AggregatorUtilTest.testOutOfOrderPruneDependentPostAgg for example.
* @param postAggName name of the postAgg on which dependency is to be calculated
*
* @return the list of dependent postAggregators
*/
public static List<PostAggregator> pruneDependentPostAgg(List<PostAggregator> postAggregatorList, String postAggName)
{

View File

@ -136,6 +136,8 @@ public class Histogram
* Returns a visual representation of a histogram object.
* Initially returns an array of just the min. and max. values
* but can also support the addition of quantiles.
*
* @return a visual representation of this histogram
*/
public HistogramVisual asVisual() {
float[] visualCounts = new float[bins.length - 2];

View File

@ -32,8 +32,6 @@ import org.mozilla.javascript.Context;
import org.mozilla.javascript.ContextAction;
import org.mozilla.javascript.ContextFactory;
import org.mozilla.javascript.Function;
import org.mozilla.javascript.NativeArray;
import org.mozilla.javascript.Scriptable;
import org.mozilla.javascript.ScriptableObject;
import javax.annotation.Nullable;
@ -41,7 +39,6 @@ import java.lang.reflect.Array;
import java.nio.ByteBuffer;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;

View File

@ -28,18 +28,20 @@ import java.nio.ByteBuffer;
/**
* Implements the HyperLogLog cardinality estimator described in:
* <p/>
*
* http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf
* <p/>
*
* Run this code to see a simple indication of expected errors based on different m values:
* <p/>
* for (int i = 1; i < 20; ++i) {
* System.out.printf("i[%,d], val[%,d] => error[%f%%]%n", i, 2 << i, 104 / Math.sqrt(2 << i));
* }
* <p/>
*
* <code>
* for (int i = 1; i &lt; 20; ++i) {
* System.out.printf("i[%,d], val[%,d] =&gt; error[%f%%]%n", i, 2 &lt;&lt; i, 104 / Math.sqrt(2 &lt;&lt; i));
* }
* </code>
*
* This class is *not* multi-threaded. It can be passed among threads, but it is written with the assumption that
* only one thread is ever calling methods on it.
* <p/>
*
* If you have multiple threads calling methods on this concurrently, I hope you manage to get correct behavior
*/
public abstract class HyperLogLogCollector implements Comparable<HyperLogLogCollector>

View File

@ -22,7 +22,6 @@ package io.druid.query.groupby;
import com.google.common.base.Function;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
@ -30,7 +29,6 @@ import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.Pair;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.ExecutorExecutingSequence;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;
@ -49,7 +47,6 @@ import io.druid.segment.StorageAdapter;
import io.druid.segment.incremental.IncrementalIndex;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;

View File

@ -23,13 +23,12 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Bytes;
import io.druid.data.input.Row;
import io.druid.query.Result;
import java.nio.ByteBuffer;
import java.util.Arrays;
/**
* The ">" operator in a "having" clause. This is similar to SQL's "having aggregation > value",
* The "&gt;" operator in a "having" clause. This is similar to SQL's "having aggregation &gt; value",
* except that an aggregation in SQL is an expression instead of an aggregation name as in Druid.
*/
public class GreaterThanHavingSpec implements HavingSpec

View File

@ -22,13 +22,12 @@ package io.druid.query.groupby.having;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Bytes;
import io.druid.data.input.Row;
import io.druid.query.Result;
import java.nio.ByteBuffer;
import java.util.Arrays;
/**
* The "<" operator in a "having" clause. This is similar to SQL's "having aggregation < value",
* The "&lt;" operator in a "having" clause. This is similar to SQL's "having aggregation &lt; value",
* except that an aggregation in SQL is an expression instead of an aggregation name as in Druid.
*/
public class LessThanHavingSpec implements HavingSpec

View File

@ -22,7 +22,6 @@ package io.druid.query.groupby.having;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.data.input.Row;
import io.druid.query.Result;
import java.nio.ByteBuffer;

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import io.druid.data.input.Row;
import io.druid.query.Result;
import java.nio.ByteBuffer;
import java.util.List;

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Function;
import com.metamx.common.guava.Sequence;
import io.druid.data.input.Row;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;

View File

@ -23,7 +23,6 @@ import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.metamx.common.guava.Sequence;
import io.druid.data.input.Row;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;

View File

@ -19,14 +19,11 @@
package io.druid.query.select;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequence;
import io.druid.query.ChainedExecutionQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryConfig;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;

View File

@ -22,7 +22,6 @@ package io.druid.query.timeboundary;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.query.BaseQuery;

View File

@ -42,7 +42,6 @@ import io.druid.query.aggregation.MetricManipulationFn;
import io.druid.timeline.LogicalSegment;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.List;

View File

@ -21,7 +21,6 @@ package io.druid.query.topn;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Ordering;
import com.metamx.common.guava.Comparators;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;

View File

@ -24,8 +24,6 @@ import io.druid.query.Result;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import java.util.Comparator;
public class TopNMapFn implements Function<Cursor, Result<TopNResultValue>>
{
private final TopNQuery query;

View File

@ -39,12 +39,12 @@ import java.util.Map;
/**
* A Builder for TopNQuery.
* <p/>
*
* Required: dataSource(), intervals(), metric() and threshold() must be called before build()
* Additional requirement for numeric metric sorts: aggregators() must be called before build()
* <p/>
*
* Optional: filters(), granularity(), postAggregators() and context() can be called before build()
* <p/>
*
* Usage example:
* <pre><code>
* TopNQuery query = new TopNQueryBuilder()

View File

@ -23,7 +23,6 @@ import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicates;
import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;
@ -38,7 +37,6 @@ import io.druid.segment.StorageAdapter;
import io.druid.segment.filter.Filters;
import org.joda.time.Interval;
import javax.sql.rowset.Predicate;
import java.nio.ByteBuffer;
import java.util.List;

View File

@ -21,7 +21,6 @@ package io.druid.query.topn;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.guava.BaseSequence;
import com.metamx.common.guava.Sequence;
import io.druid.collections.StupidPool;
import io.druid.guice.annotations.Global;
@ -35,7 +34,6 @@ import io.druid.query.Result;
import io.druid.segment.Segment;
import java.nio.ByteBuffer;
import java.util.Iterator;
import java.util.concurrent.ExecutorService;
/**

View File

@ -20,11 +20,8 @@
package io.druid.query.topn;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import java.util.Iterator;
import java.util.List;
/**
*/

View File

@ -42,7 +42,7 @@ public interface DimensionSelector
*
* Value cardinality would be 2.
*
* @return
* @return the value cardinality
*/
public int getValueCardinality();
@ -57,26 +57,26 @@ public interface DimensionSelector
*
* getRow() would return
*
* getRow(0) => [0 1]
* getRow(1) => [0]
* getRow(2) => [0 1]
* getRow(3) => [1]
* getRow(0) =&gt; [0 1]
* getRow(1) =&gt; [0]
* getRow(2) =&gt; [0 1]
* getRow(3) =&gt; [1]
*
* and then lookupName would return:
*
* lookupName(0) => A
* lookupName(1) => B
* lookupName(0) =&gt; A
* lookupName(1) =&gt; B
*
* @param id
* @return
* @param id id to lookup the field name for
* @return the field name for the given id
*/
public String lookupName(int id);
/**
* The ID is the int id value of the field.
*
* @param name
* @return
* @param name field name to look up the id for
* @return the id for the given field name
*/
public int lookupId(String name);
}

View File

@ -29,7 +29,6 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.Files;
import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints;
@ -51,7 +50,6 @@ import io.druid.common.utils.JodaUtils;
import io.druid.common.utils.SerializerUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.ByteBufferWriter;
import io.druid.segment.data.CompressedLongsSupplierSerializer;
import io.druid.segment.data.ConciseCompressedIndexedInts;
@ -114,7 +112,8 @@ public class IndexMerger
* @param dataInterval the Interval that the data represents
* @param outDir the directory to persist the data to
*
* @throws java.io.IOException
* @return the index output directory
* @throws java.io.IOException if an IO error occurs persisting the index
*/
public static File persist(final IncrementalIndex index, final Interval dataInterval, File outDir) throws IOException
{

View File

@ -33,7 +33,7 @@ public interface QueryableIndex extends ColumnSelector
/**
* The close method shouldn't actually be here as this is nasty. We will adjust it in the future.
* @throws java.io.IOException
* @throws java.io.IOException if an exception was thrown closing the index
*/
@Deprecated
public void close() throws IOException;

View File

@ -20,7 +20,6 @@
package io.druid.segment.data;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;

View File

@ -20,7 +20,6 @@
package io.druid.segment.data;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;

View File

@ -20,7 +20,6 @@
package io.druid.segment.data;
import com.google.common.base.Charsets;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import com.metamx.common.IAE;
@ -41,13 +40,13 @@ import java.util.Map;
/**
* A generic, flat storage mechanism. Use static methods fromArray() or fromIterable() to construct. If input
* is sorted, supports binary search index lookups. If input is not sorted, only supports array-like index lookups.
* <p/>
*
* V1 Storage Format:
* <p/>
*
* byte 1: version (0x1)
* byte 2 == 0x1 => allowReverseLookup
* bytes 3-6 => numBytesUsed
* bytes 7-10 => numElements
* byte 2 == 0x1 =&gt; allowReverseLookup
* bytes 3-6 =&gt; numBytesUsed
* bytes 7-10 =&gt; numElements
* bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values
* bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes for value
*/
@ -327,8 +326,9 @@ public class GenericIndexed<T> implements Indexed<T>, Closeable
/**
* The returned GenericIndexed must be closed to release the underlying memory
* @param maxBytes
* @return
*
* @param maxBytes maximum size in bytes of the lookup cache
* @return a copy of this GenericIndexed with a lookup cache.
*/
public GenericIndexed<T> withCache(int maxBytes)
{

View File

@ -19,9 +19,6 @@
package io.druid.segment.data;
/**
* @param <T>
*/
public interface Indexed<T> extends Iterable<T>
{
Class<? extends T> getClazz();

View File

@ -34,7 +34,7 @@ public interface ObjectStrategy<T> extends Comparator<T>
*
* @param buffer buffer to read value from
* @param numBytes number of bytes used to store the value, starting at buffer.position()
* @return
* @return an object created from the given byte buffer representation
*/
public T fromByteBuffer(ByteBuffer buffer, int numBytes);
public byte[] toBytes(T val);

View File

@ -144,7 +144,7 @@ public class IncrementalIndex implements Iterable<Row>
/**
* Adds a new row. The row might correspond with another row that already exists, in which case this will
* update that row instead of inserting a new one.
* <p/>
*
*
* Calls to add() are thread safe.
*

View File

@ -40,6 +40,7 @@ public abstract class ComplexMetricSerde
* added to the builder.
*
* @param buffer the buffer to deserialize
* @param builder ColumnBuilder to add the column to
* @return a ColumnPartSerde that can serialize out the object that was read from the buffer to the builder
*/
public abstract ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder);

View File

@ -21,7 +21,6 @@ package io.druid.data.input;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.JSONParseSpec;
import io.druid.data.input.impl.ParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import org.joda.time.DateTime;
import org.junit.Test;

View File

@ -27,7 +27,6 @@ import org.junit.Before;
import org.junit.Test;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;

View File

@ -1,7 +1,6 @@
package io.druid.query;
import com.google.common.base.Supplier;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.collections.StupidPool;
import io.druid.query.search.SearchQueryQueryToolChest;
import io.druid.query.search.SearchQueryRunnerFactory;

View File

@ -24,12 +24,10 @@ package io.druid.query.groupby;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.Druids;
import io.druid.query.Query;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import org.junit.Assert;

View File

@ -26,7 +26,6 @@ import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.Row;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.Result;
import org.junit.Test;
import java.util.ArrayList;

View File

@ -23,13 +23,10 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.QueryWatcher;
import io.druid.query.Result;
import io.druid.query.filter.DimFilter;
import io.druid.query.search.search.FragmentSearchQuerySpec;

View File

@ -22,15 +22,12 @@ package io.druid.query.select;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequences;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.Query;
import io.druid.query.QueryConfig;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.QueryWatcher;
import io.druid.query.Result;
import io.druid.query.TableDataSource;
import io.druid.query.filter.SelectorDimFilter;

View File

@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.metamx.common.Granularity;
import com.metamx.common.guava.Sequences;
import io.druid.granularity.PeriodGranularity;
import io.druid.granularity.QueryGranularity;

View File

@ -23,7 +23,6 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexAdapter;
import org.joda.time.Interval;

View File

@ -26,7 +26,6 @@ import com.google.common.io.Files;
import io.druid.data.input.MapBasedInputRow;
import io.druid.granularity.QueryGranularity;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.IncrementalIndexTest;
import io.druid.segment.incremental.IncrementalIndex;
import junit.framework.Assert;

View File

@ -49,7 +49,6 @@ import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment;
import io.druid.segment.TestHelper;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import org.joda.time.DateTime;

View File

@ -49,7 +49,6 @@ import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment;
import io.druid.segment.TestHelper;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import org.joda.time.DateTime;

View File

@ -52,7 +52,6 @@ import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
/**
*/

View File

@ -22,16 +22,16 @@ package io.druid.firehose.rabbitmq;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.common.logger.Logger;
import com.rabbitmq.client.AMQP;
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
import com.rabbitmq.client.ConnectionFactory;
import com.rabbitmq.client.ConsumerCancelledException;
import com.rabbitmq.client.DefaultConsumer;
import com.rabbitmq.client.Envelope;
import com.rabbitmq.client.AMQP;
import com.rabbitmq.client.QueueingConsumer.Delivery;
import com.rabbitmq.client.ShutdownListener;
import com.rabbitmq.client.ShutdownSignalException;
import com.rabbitmq.client.ConsumerCancelledException;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
@ -49,14 +49,14 @@ import java.util.concurrent.LinkedBlockingQueue;
/**
* A FirehoseFactory for RabbitMQ.
* <p/>
*
* It will receive it's configuration through the realtime.spec file and expects to find a
* consumerProps element in the firehose definition with values for a number of configuration options.
* Below is a complete example for a RabbitMQ firehose configuration with some explanation. Options
* that have defaults can be skipped but options with no defaults must be specified with the exception
* of the URI property. If the URI property is set, it will override any other property that was also
* set.
* <p/>
*
* File: <em>realtime.spec</em>
* <pre>
* "firehose" : {
@ -88,7 +88,7 @@ import java.util.concurrent.LinkedBlockingQueue;
* }
* },
* </pre>
* <p/>
*
* <b>Limitations:</b> This implementation will not attempt to reconnect to the MQ broker if the
* connection to it is lost. Furthermore it does not support any automatic failover on high availability
* RabbitMQ clusters. This is not supported by the underlying AMQP client library and while the behavior
@ -96,7 +96,7 @@ import java.util.concurrent.LinkedBlockingQueue;
* the RabbitMQ cluster that sets the "ha-mode" and "ha-sync-mode" properly on the queue that this
* Firehose connects to, messages should survive an MQ broker node failure and be delivered once a
* connection to another node is set up.
* <p/>
*
* For more information on RabbitMQ high availability please see:
* <a href="http://www.rabbitmq.com/ha.html">http://www.rabbitmq.com/ha.html</a>.
*/

View File

@ -27,9 +27,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.impl.FileIteratingFirehose;

View File

@ -22,16 +22,12 @@ package io.druid.storage.s3;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSSessionCredentials;
import com.google.common.base.Charsets;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;

Some files were not shown because too many files have changed in this diff Show More