*/
IntIterator descendingIterator();
- /**
- * Prints debug info about the given {@link IntSet} implementation
- *
- * @return a string that describes the internal representation of the
- * instance
- */
- String debugInfo();
-
- /**
- * Adds to the set all the elements between first and
- * last, both included.
- *
- * @param from first element
- * @param to last element
- */
- void fill(int from, int to);
-
- /**
- * Removes from the set all the elements between first and
- * last, both included.
- *
- * @param from first element
- * @param to last element
- */
- void clear(int from, int to);
-
- /**
- * Adds the element if it not existing, or removes it if existing
- *
- * @param e element to flip
- */
- void flip(int e);
-
- /**
- * Gets the ith element of the set
- *
- * @param i position of the element in the sorted set
- *
- * @return the ith element of the set
- *
- * @throws IndexOutOfBoundsException if i is less than zero, or greater or equal to
- * {@link #size()}
- */
- int get(int i);
-
- /**
- * Provides position of element within the set.
- *
- * It returns -1 if the element does not exist within the set.
- *
- * @param e element of the set
- *
- * @return the element position
- */
- int indexOf(int e);
-
- /**
- * Converts a given array into an instance of the current class.
- *
- * @param a array to use to generate the new instance
- *
- * @return the converted collection
- */
- IntSet convert(int... a);
-
- /**
- * Converts a given collection into an instance of the current class.
- *
- * @param c array to use to generate the new instance
- *
- * @return the converted collection
- */
- IntSet convert(Collection c);
-
- /**
- * Returns the first (lowest) element currently in this set.
- *
- * @return the first (lowest) element currently in this set
- *
- * @throws NoSuchElementException if this set is empty
- */
- int first();
-
- /**
- * Returns the last (highest) element currently in this set.
- *
- * @return the last (highest) element currently in this set
- *
- * @throws NoSuchElementException if this set is empty
- */
- int last();
-
/**
* @return the number of elements in this set (its cardinality)
*/
@@ -219,120 +57,6 @@ public interface IntSet extends Cloneable, Comparable
*/
boolean isEmpty();
- /**
- * Returns true if this set contains the specified element.
- *
- * @param i element whose presence in this set is to be tested
- *
- * @return true if this set contains the specified element
- */
- boolean contains(int i);
-
- /**
- * Adds the specified element to this set if it is not already present. It
- * ensures that sets never contain duplicate elements.
- *
- * @param i element to be added to this set
- *
- * @return true if this set did not already contain the specified
- * element
- *
- * @throws IllegalArgumentException if some property of the specified element prevents it from
- * being added to this set
- */
- boolean add(int i);
-
- /**
- * Removes the specified element from this set if it is present.
- *
- * @param i object to be removed from this set, if present
- *
- * @return true if this set contained the specified element
- *
- * @throws UnsupportedOperationException if the remove operation is not supported by this set
- */
- boolean remove(int i);
-
- /**
- * Adds all of the elements in the specified collection to this set if
- * they're not already present.
- *
- * @param c collection containing elements to be added to this set
- *
- * @return true if this set changed as a result of the call
- *
- * @throws NullPointerException if the specified collection contains one or more null
- * elements and this set does not permit null elements, or if
- * the specified collection is null
- * @throws IllegalArgumentException if some property of an element of the specified collection
- * prevents it from being added to this set
- * @see #add(int)
- */
- boolean addAll(IntSet c);
-
- /**
- * Retains only the elements in this set that are contained in the specified
- * collection. In other words, removes from this set all of its elements
- * that are not contained in the specified collection.
- *
- * @param c collection containing elements to be retained in this set
- *
- * @return true if this set changed as a result of the call
- *
- * @throws NullPointerException if this set contains a null element and the specified
- * collection does not permit null elements (optional), or if
- * the specified collection is null
- * @see #remove(int)
- */
- boolean retainAll(IntSet c);
-
- /**
- * Removes from this set all of its elements that are contained in the
- * specified collection.
- *
- * @param c collection containing elements to be removed from this set
- *
- * @return true if this set changed as a result of the call
- *
- * @throws NullPointerException if this set contains a null element and the specified
- * collection does not permit null elements (optional), or if
- * the specified collection is null
- * @see #remove(int)
- * @see #contains(int)
- */
- boolean removeAll(IntSet c);
-
- /**
- * Removes all of the elements from this set. The set will be empty after
- * this call returns.
- *
- * @throws UnsupportedOperationException if the clear method is not supported by this set
- */
- void clear();
-
- /**
- * @return an array containing all the elements in this set, in the same
- * order.
- */
- int[] toArray();
-
- /**
- * Returns an array containing all of the elements in this set.
- *
- * If this set fits in the specified array with room to spare (i.e., the
- * array has more elements than this set), the element in the array
- * immediately following the end of the set are left unchanged.
- *
- * @param a the array into which the elements of this set are to be
- * stored.
- *
- * @return the array containing all the elements in this set
- *
- * @throws NullPointerException if the specified array is null
- * @throws IllegalArgumentException if this set does not fit in the specified array
- */
- int[] toArray(int[] a);
-
/**
* An {@link Iterator}-like interface that allows to "skip" some elements of
* the set
@@ -353,22 +77,6 @@ interface IntIterator extends org.roaringbitmap.IntIterator
@Override
int next();
- /**
- * Removes from the underlying collection the last element returned by
- * the iterator (optional operation). This method can be called only
- * once per call to next . The behavior of an iterator is
- * unspecified if the underlying collection is modified while the
- * iteration is in progress in any way other than by calling this
- * method.
- *
- * @throws UnsupportedOperationException if the remove operation is not supported by
- * this Iterator.
- * @throws IllegalStateException if the next method has not yet been called,
- * or the remove method has already been called
- * after the last call to the next method.
- */
- void remove();
-
/**
* Skips all the elements before the the specified element, so that
* {@link #next()} gives the given element or, if it does not exist, the
diff --git a/extendedset/src/main/java/io/druid/extendedset/utilities/IntList.java b/extendedset/src/main/java/io/druid/extendedset/utilities/IntList.java
index 501dbb362a46..c667e6076bb5 100755
--- a/extendedset/src/main/java/io/druid/extendedset/utilities/IntList.java
+++ b/extendedset/src/main/java/io/druid/extendedset/utilities/IntList.java
@@ -1,6 +1,5 @@
package io.druid.extendedset.utilities;
-import java.nio.IntBuffer;
import java.util.ArrayList;
/**
@@ -68,27 +67,6 @@ public int get(int index)
return baseList[index % ALLOCATION_SIZE];
}
- public int baseListCount()
- {
- return baseLists.size();
- }
-
- public IntBuffer getBaseList(int index)
- {
- final int[] array = baseLists.get(index);
- if (array == null) {
- return null;
- }
-
- final IntBuffer retVal = IntBuffer.wrap(array);
-
- if (index + 1 == baseListCount()) {
- retVal.limit(maxIndex - (index * ALLOCATION_SIZE));
- }
-
- return retVal.asReadOnlyBuffer();
- }
-
public int[] toArray()
{
int[] retVal = new int[length()];
diff --git a/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetIntersectionTest.java b/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetIntersectionTest.java
index 221099e82bb8..4493312d0118 100644
--- a/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetIntersectionTest.java
+++ b/extendedset/src/test/java/io/druid/extendedset/intset/ImmutableConciseSetIntersectionTest.java
@@ -548,7 +548,7 @@ public void testIntersectionFirstOneFillRemovedFromQueue()
}
@Test
- public void testIntersectionTerminates() throws Exception
+ public void testIntersectionTerminates()
{
verifyIntersection(Collections.emptyList(), Arrays.asList(new ImmutableConciseSet(), new ImmutableConciseSet()));
}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
index f8f2e76fa1fc..aedb8f10553a 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
@@ -21,17 +21,16 @@
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.emitter.core.Emitter;
import io.druid.java.util.emitter.core.Event;
import io.druid.java.util.emitter.service.AlertEvent;
import io.druid.java.util.emitter.service.ServiceMetricEvent;
-import io.druid.java.util.common.StringUtils;
-import io.druid.java.util.common.logger.Logger;
import org.apache.hadoop.metrics2.sink.timeline.AbstractTimelineMetricsSink;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
-import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
@@ -202,7 +201,7 @@ public void run()
}
@Override
- public void flush() throws IOException
+ public void flush()
{
synchronized (started) {
if (started.get()) {
@@ -220,7 +219,7 @@ public void flush() throws IOException
}
@Override
- public void close() throws IOException
+ public void close()
{
synchronized (started) {
flush();
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
index 18368cf92702..ec2321989da0 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/test/java/io/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
@@ -20,9 +20,10 @@
package io.druid.emitter.ambari.metrics;
import com.google.common.collect.Maps;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
+import io.druid.annotations.UsedByJUnitParamsRunner;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
import junitparams.JUnitParamsRunner;
import junitparams.Parameters;
import org.apache.commons.io.IOUtils;
@@ -44,12 +45,8 @@
public class WhiteListBasedDruidToTimelineEventConverterTest
{
private final String prefix = "druid";
- private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter = new WhiteListBasedDruidToTimelineEventConverter(
- prefix,
- "druid",
- null,
- new DefaultObjectMapper()
- );
+ private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter =
+ new WhiteListBasedDruidToTimelineEventConverter(prefix, "druid", null, new DefaultObjectMapper());
private ServiceMetricEvent event;
private final DateTime createdTime = DateTimes.nowUtc();
private final String hostname = "testHost:8080";
@@ -143,6 +140,7 @@ public void testWhiteListedStringArrayDimension() throws IOException
Assert.assertEquals(defaultNamespace + ".g1.jvm/gc/cpu", metric.getMetricName());
}
+ @UsedByJUnitParamsRunner
private Object[] parametersForTestGetName()
{
return new Object[]{
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentKiller.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentKiller.java
index 9c4a6007b4e6..42922dcb226b 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentKiller.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentKiller.java
@@ -21,14 +21,12 @@
import com.google.inject.Inject;
import com.microsoft.azure.storage.StorageException;
-
import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
-import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.file.Paths;
import java.util.Map;
@@ -69,7 +67,7 @@ public void kill(DataSegment segment) throws SegmentLoadingException
}
@Override
- public void killAll() throws IOException
+ public void killAll()
{
throw new UnsupportedOperationException("not implemented");
}
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPuller.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPuller.java
index ebdb79bbebc3..eedef9703b05 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPuller.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureDataSegmentPuller.java
@@ -19,22 +19,17 @@
package io.druid.storage.azure;
-import com.google.common.annotations.VisibleForTesting;
import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import io.druid.java.util.common.CompressionUtils;
-import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.logger.Logger;
-import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
-import io.druid.timeline.DataSegment;
import org.apache.commons.io.FileUtils;
import java.io.File;
import java.io.IOException;
-import java.util.Map;
-public class AzureDataSegmentPuller implements DataSegmentPuller
+public class AzureDataSegmentPuller
{
private static final Logger log = new Logger(AzureDataSegmentPuller.class);
@@ -55,7 +50,7 @@ public AzureDataSegmentPuller(
this.azureStorage = azureStorage;
}
- public io.druid.java.util.common.FileUtils.FileCopyResult getSegmentFiles(
+ io.druid.java.util.common.FileUtils.FileCopyResult getSegmentFiles(
final String containerName,
final String blobPath,
final File outDir
@@ -63,7 +58,7 @@ public io.druid.java.util.common.FileUtils.FileCopyResult getSegmentFiles(
throws SegmentLoadingException
{
try {
- prepareOutDir(outDir);
+ FileUtils.forceMkdir(outDir);
log.info(
"Loading container: [%s], with blobPath: [%s] and outDir: [%s]", containerName, blobPath, outDir
@@ -104,23 +99,5 @@ public io.druid.java.util.common.FileUtils.FileCopyResult getSegmentFiles(
}
throw new SegmentLoadingException(e, e.getMessage());
}
-
- }
-
- @Override
- public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException
- {
-
- final Map loadSpec = segment.getLoadSpec();
- final String containerName = MapUtils.getString(loadSpec, "containerName");
- final String blobPath = MapUtils.getString(loadSpec, "blobPath");
-
- getSegmentFiles(containerName, blobPath, outDir);
- }
-
- @VisibleForTesting
- void prepareOutDir(final File outDir) throws IOException
- {
- FileUtils.forceMkdir(outDir);
}
}
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorageDruidModule.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorageDruidModule.java
index 0ed18ae1d5cb..a6cda5499e64 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorageDruidModule.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureStorageDruidModule.java
@@ -81,10 +81,6 @@ public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.azure", AzureAccountConfig.class);
- Binders.dataSegmentPullerBinder(binder)
- .addBinding(SCHEME)
- .to(AzureDataSegmentPuller.class).in(LazySingleton.class);
-
Binders.dataSegmentPusherBinder(binder)
.addBinding(SCHEME)
.to(AzureDataSegmentPusher.class).in(LazySingleton.class);
diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java
index 9b9ceff44c2c..2cd17c1c41c1 100644
--- a/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java
+++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/storage/azure/AzureTaskLogs.java
@@ -50,7 +50,7 @@ public AzureTaskLogs(AzureTaskLogsConfig config, AzureStorage azureStorage)
}
@Override
- public void pushTaskLog(final String taskid, final File logFile) throws IOException
+ public void pushTaskLog(final String taskid, final File logFile)
{
final String taskKey = getTaskLogKey(taskid);
log.info("Pushing task log %s to: %s", logFile, taskKey);
@@ -123,13 +123,13 @@ private String getTaskLogKey(String taskid)
}
@Override
- public void killAll() throws IOException
+ public void killAll()
{
throw new UnsupportedOperationException("not implemented");
}
@Override
- public void killOlderThan(long timestamp) throws IOException
+ public void killOlderThan(long timestamp)
{
throw new UnsupportedOperationException("not implemented");
}
diff --git a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPullerTest.java b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPullerTest.java
index b01e6b638bd4..cc456d532971 100644
--- a/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPullerTest.java
+++ b/extensions-contrib/azure-extensions/src/test/java/io/druid/storage/azure/AzureDataSegmentPullerTest.java
@@ -19,13 +19,9 @@
package io.druid.storage.azure;
-import com.google.common.collect.ImmutableMap;
import com.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.FileUtils;
-import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
-import io.druid.timeline.DataSegment;
-import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport;
import org.junit.Before;
import org.junit.Test;
@@ -48,17 +44,6 @@ public class AzureDataSegmentPullerTest extends EasyMockSupport
private static final String SEGMENT_FILE_NAME = "segment";
private static final String containerName = "container";
private static final String blobPath = "/path/to/storage/index.zip";
- private static final DataSegment dataSegment = new DataSegment(
- "test",
- Intervals.of("2015-04-12/2015-04-13"),
- "1",
- ImmutableMap.of("containerName", containerName, "blobPath", blobPath),
- null,
- null,
- NoneShardSpec.instance(),
- 0,
- 1
- );
private AzureStorage azureStorage;
@Before
@@ -129,44 +114,4 @@ public void testDeleteOutputDirectoryWhenErrorIsRaisedPullingSegmentFiles()
}
}
-
- @Test
- public void getSegmentFilesTest() throws SegmentLoadingException
- {
- final File outDir = new File("");
- try {
- final FileUtils.FileCopyResult result = createMock(FileUtils.FileCopyResult.class);
- final AzureDataSegmentPuller puller = createMockBuilder(AzureDataSegmentPuller.class).withConstructor(
- azureStorage
- ).addMockedMethod("getSegmentFiles", String.class, String.class, File.class).createMock();
-
- expect(puller.getSegmentFiles(containerName, blobPath, outDir)).andReturn(result);
-
- replayAll();
-
- puller.getSegmentFiles(dataSegment, outDir);
-
- verifyAll();
- }
- finally {
- outDir.delete();
- }
-
- }
-
- @Test
- public void prepareOutDirTest() throws IOException
- {
- File outDir = Files.createTempDirectory("druid").toFile();
-
- try {
- AzureDataSegmentPuller puller = new AzureDataSegmentPuller(azureStorage);
- puller.prepareOutDir(outDir);
-
- assertTrue(outDir.exists());
- }
- finally {
- outDir.delete();
- }
- }
}
diff --git a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java
index 80ad7dab7ab3..6bbcecbb8613 100644
--- a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java
+++ b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java
@@ -24,12 +24,9 @@
import com.netflix.astyanax.recipes.storage.ChunkedStorage;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.FileUtils;
-import io.druid.java.util.common.ISE;
import io.druid.java.util.common.RetryUtils;
import io.druid.java.util.common.logger.Logger;
-import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
-import io.druid.timeline.DataSegment;
import java.io.File;
import java.io.FileOutputStream;
@@ -39,7 +36,7 @@
/**
* Cassandra Segment Puller
*/
-public class CassandraDataSegmentPuller extends CassandraStorage implements DataSegmentPuller
+public class CassandraDataSegmentPuller extends CassandraStorage
{
private static final Logger log = new Logger(CassandraDataSegmentPuller.class);
private static final int CONCURRENCY = 10;
@@ -51,22 +48,14 @@ public CassandraDataSegmentPuller(CassandraDataSegmentConfig config)
super(config);
}
- @Override
- public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException
- {
- String key = (String) segment.getLoadSpec().get("key");
- getSegmentFiles(key, outDir);
- }
- public FileUtils.FileCopyResult getSegmentFiles(final String key, final File outDir)
- throws SegmentLoadingException
+ FileUtils.FileCopyResult getSegmentFiles(final String key, final File outDir) throws SegmentLoadingException
{
log.info("Pulling index from C* at path[%s] to outDir[%s]", key, outDir);
- if (!outDir.exists()) {
- outDir.mkdirs();
+ try {
+ org.apache.commons.io.FileUtils.forceMkdir(outDir);
}
-
- if (!outDir.isDirectory()) {
- throw new ISE("outDir[%s] must be a directory.", outDir);
+ catch (IOException e) {
+ throw new SegmentLoadingException(e, "");
}
long startTime = System.currentTimeMillis();
diff --git a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java
index a3f61a07f9dc..885c03a21cae 100644
--- a/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java
+++ b/extensions-contrib/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java
@@ -24,7 +24,6 @@
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Key;
-import io.druid.guice.Binders;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
import io.druid.guice.PolyBind;
@@ -42,11 +41,6 @@ public class CassandraDruidModule implements DruidModule
@Override
public void configure(Binder binder)
{
- Binders.dataSegmentPullerBinder(binder)
- .addBinding(SCHEME)
- .to(CassandraDataSegmentPuller.class)
- .in(LazySingleton.class);
-
PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class))
.addBinding(SCHEME)
.to(CassandraDataSegmentPusher.class)
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java
index 6a3fa6414728..d7f05de854d5 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java
@@ -20,22 +20,17 @@
package io.druid.storage.cloudfiles;
import com.google.inject.Inject;
-
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.ISE;
-import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.logger.Logger;
-import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
-import io.druid.timeline.DataSegment;
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import java.io.File;
import java.io.IOException;
-import java.util.Map;
-public class CloudFilesDataSegmentPuller implements DataSegmentPuller
+public class CloudFilesDataSegmentPuller
{
private static final Logger log = new Logger(CloudFilesDataSegmentPuller.class);
@@ -47,20 +42,7 @@ public CloudFilesDataSegmentPuller(final CloudFilesApi cloudFilesApi)
this.cloudFilesApi = cloudFilesApi;
}
- @Override
- public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException
- {
- final Map loadSpec = segment.getLoadSpec();
- final String region = MapUtils.getString(loadSpec, "region");
- final String container = MapUtils.getString(loadSpec, "container");
- final String path = MapUtils.getString(loadSpec, "path");
-
- log.info("Pulling index at path[%s] to outDir[%s]", path, outDir);
- prepareOutDir(outDir);
- getSegmentFiles(region, container, path, outDir);
- }
-
- public FileUtils.FileCopyResult getSegmentFiles(String region, String container, String path, File outDir)
+ FileUtils.FileCopyResult getSegmentFiles(String region, String container, String path, File outDir)
throws SegmentLoadingException
{
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
@@ -68,8 +50,10 @@ public FileUtils.FileCopyResult getSegmentFiles(String region, String container,
try {
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
- byteSource, outDir,
- CloudFilesUtils.CLOUDFILESRETRY, false
+ byteSource,
+ outDir,
+ CloudFilesUtils.CLOUDFILESRETRY,
+ false
);
log.info("Loaded %d bytes from [%s] to [%s]", result.size(), path, outDir.getAbsolutePath());
return result;
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
index 8edabff17597..98b3c9b2a033 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
@@ -31,7 +31,6 @@
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import java.io.File;
-import java.io.IOException;
import java.net.URI;
import java.nio.file.Files;
import java.util.Map;
@@ -74,7 +73,6 @@ public String getPathForHadoop(final String dataSource)
@Override
public DataSegment push(final File indexFilesDir, final DataSegment inSegment, final boolean replaceExisting)
- throws IOException
{
final String segmentPath = CloudFilesUtils.buildCloudFilesPath(this.config.getBasePath(), getStorageDir(inSegment));
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
index 7e87aa02db88..6597a910f9d7 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
@@ -25,13 +25,11 @@
import com.google.common.collect.ImmutableSet;
import com.google.inject.Binder;
import com.google.inject.Provides;
-
import io.druid.guice.Binders;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
import io.druid.initialization.DruidModule;
import io.druid.java.util.common.logger.Logger;
-
import org.jclouds.ContextBuilder;
import org.jclouds.logging.slf4j.config.SLF4JLoggingModule;
import org.jclouds.openstack.v2_0.config.InternalUrlModule;
@@ -86,8 +84,6 @@ public void configure(Binder binder)
JsonConfigProvider.bind(binder, "druid.storage", CloudFilesDataSegmentPusherConfig.class);
JsonConfigProvider.bind(binder, "druid.cloudfiles", CloudFilesAccountConfig.class);
- Binders.dataSegmentPullerBinder(binder).addBinding(SCHEME).to(CloudFilesDataSegmentPuller.class)
- .in(LazySingleton.class);
Binders.dataSegmentPusherBinder(binder).addBinding(SCHEME).to(CloudFilesDataSegmentPusher.class)
.in(LazySingleton.class);
diff --git a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java
index 1f4f03cb9594..c942b1d037d0 100644
--- a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java
+++ b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java
@@ -310,7 +310,7 @@ public void run()
}
@Override
- public void close() throws IOException
+ public void close()
{
defaultMQPullConsumer.shutdown();
pullMessageService.shutdown(false);
diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java
index b7fbdc162a23..5a476a1170b8 100644
--- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java
+++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentKiller.java
@@ -61,7 +61,7 @@ public void kill(DataSegment segment) throws SegmentLoadingException
}
@Override
- public void killAll() throws IOException
+ public void killAll()
{
throw new UnsupportedOperationException("not implemented");
}
diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPuller.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPuller.java
index 8d1dc66fd9fe..3bec0cd0d7b4 100644
--- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPuller.java
+++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleDataSegmentPuller.java
@@ -19,25 +19,20 @@
package io.druid.storage.google;
-import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicate;
import com.google.inject.Inject;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.FileUtils;
-import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.logger.Logger;
-import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.segment.loading.URIDataPuller;
-import io.druid.timeline.DataSegment;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
-import java.util.Map;
-public class GoogleDataSegmentPuller implements DataSegmentPuller, URIDataPuller
+public class GoogleDataSegmentPuller implements URIDataPuller
{
private static final Logger LOG = new Logger(GoogleDataSegmentPuller.class);
@@ -49,23 +44,13 @@ public GoogleDataSegmentPuller(final GoogleStorage storage)
this.storage = storage;
}
- @Override
- public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException
- {
- final Map loadSpec = segment.getLoadSpec();
- final String bucket = MapUtils.getString(loadSpec, "bucket");
- final String path = MapUtils.getString(loadSpec, "path");
-
- getSegmentFiles(bucket, path, outDir);
- }
-
- public FileUtils.FileCopyResult getSegmentFiles(final String bucket, final String path, File outDir)
+ FileUtils.FileCopyResult getSegmentFiles(final String bucket, final String path, File outDir)
throws SegmentLoadingException
{
LOG.info("Pulling index at bucket[%s] path[%s] to outDir[%s]", bucket, path, outDir.getAbsolutePath());
try {
- prepareOutDir(outDir);
+ org.apache.commons.io.FileUtils.forceMkdir(outDir);
final GoogleByteSource byteSource = new GoogleByteSource(storage, bucket, path);
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
@@ -91,12 +76,6 @@ public FileUtils.FileCopyResult getSegmentFiles(final String bucket, final Strin
}
}
- @VisibleForTesting
- void prepareOutDir(final File outDir) throws IOException
- {
- org.apache.commons.io.FileUtils.forceMkdir(outDir);
- }
-
@Override
public InputStream getInputStream(URI uri) throws IOException
{
diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java
index 8167e2c879f2..29c7fb939ce6 100644
--- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java
+++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java
@@ -89,8 +89,6 @@ public void configure(Binder binder)
JsonConfigProvider.bind(binder, "druid.google", GoogleAccountConfig.class);
- Binders.dataSegmentPullerBinder(binder).addBinding(SCHEME).to(GoogleDataSegmentPuller.class)
- .in(LazySingleton.class);
Binders.dataSegmentPusherBinder(binder).addBinding(SCHEME).to(GoogleDataSegmentPusher.class)
.in(LazySingleton.class);
Binders.dataSegmentKillerBinder(binder).addBinding(SCHEME).to(GoogleDataSegmentKiller.class)
diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java
index 8d0fbfd055a4..6ed64576266f 100644
--- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java
+++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleTaskLogs.java
@@ -112,13 +112,13 @@ private String getTaskLogKey(String taskid)
}
@Override
- public void killAll() throws IOException
+ public void killAll()
{
throw new UnsupportedOperationException("not implemented");
}
@Override
- public void killOlderThan(long timestamp) throws IOException
+ public void killOlderThan(long timestamp)
{
throw new UnsupportedOperationException("not implemented");
}
diff --git a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPullerTest.java b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPullerTest.java
index 1504a20333fb..1d391f2d8abd 100644
--- a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPullerTest.java
+++ b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPullerTest.java
@@ -19,12 +19,8 @@
package io.druid.storage.google;
-import com.google.common.collect.ImmutableMap;
-import io.druid.java.util.common.FileUtils;
-import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
-import io.druid.timeline.DataSegment;
-import io.druid.timeline.partition.NoneShardSpec;
+import org.apache.commons.io.FileUtils;
import org.easymock.EasyMockSupport;
import org.junit.Test;
@@ -34,23 +30,11 @@
import static org.easymock.EasyMock.expect;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
public class GoogleDataSegmentPullerTest extends EasyMockSupport
{
private static final String bucket = "bucket";
private static final String path = "/path/to/storage/index.zip";
- private static final DataSegment dataSegment = new DataSegment(
- "test",
- Intervals.of("2015-04-12/2015-04-13"),
- "1",
- ImmutableMap.of("bucket", bucket, "path", path),
- null,
- null,
- NoneShardSpec.instance(),
- 0,
- 1
- );
@Test(expected = SegmentLoadingException.class)
public void testDeleteOutputDirectoryWhenErrorIsRaisedPullingSegmentFiles()
@@ -72,48 +56,7 @@ public void testDeleteOutputDirectoryWhenErrorIsRaisedPullingSegmentFiles()
verifyAll();
}
finally {
- org.apache.commons.io.FileUtils.deleteDirectory(outDir);
- }
- }
-
- @Test
- public void getSegmentFilesTest() throws SegmentLoadingException, IOException
- {
- final File outDir = new File("");
- try {
- final FileUtils.FileCopyResult result = createMock(FileUtils.FileCopyResult.class);
- GoogleStorage storage = createMock(GoogleStorage.class);
- GoogleDataSegmentPuller puller = createMockBuilder(GoogleDataSegmentPuller.class).withConstructor(
- storage
- ).addMockedMethod("getSegmentFiles", String.class, String.class, File.class).createMock();
-
- expect(puller.getSegmentFiles(bucket, path, outDir)).andReturn(result);
-
- replayAll();
-
- puller.getSegmentFiles(dataSegment, outDir);
-
- verifyAll();
- }
- finally {
- org.apache.commons.io.FileUtils.deleteDirectory(outDir);
- }
- }
-
- @Test
- public void prepareOutDirTest() throws IOException
- {
- GoogleStorage storage = createMock(GoogleStorage.class);
- File outDir = Files.createTempDirectory("druid").toFile();
-
- try {
- GoogleDataSegmentPuller puller = new GoogleDataSegmentPuller(storage);
- puller.prepareOutDir(outDir);
-
- assertTrue(outDir.exists());
- }
- finally {
- org.apache.commons.io.FileUtils.deleteDirectory(outDir);
+ FileUtils.deleteDirectory(outDir);
}
}
}
diff --git a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java
index 11372c243aa0..53d753a5b877 100644
--- a/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java
+++ b/extensions-contrib/google-extensions/src/test/java/io/druid/storage/google/GoogleDataSegmentPusherTest.java
@@ -48,17 +48,6 @@ public class GoogleDataSegmentPusherTest extends EasyMockSupport
private static final String bucket = "bucket";
private static final String prefix = "prefix";
private static final String path = "prefix/test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip";
- private static final DataSegment dataSegment = new DataSegment(
- "test",
- Intervals.of("2015-04-12/2015-04-13"),
- "1",
- ImmutableMap.of("bucket", bucket, "path", path),
- null,
- null,
- new NoneShardSpec(),
- 0,
- 1
- );
private GoogleStorage storage;
private GoogleAccountConfig googleAccountConfig;
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java
index bba2796862c5..22293905adda 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/GraphiteEmitter.java
@@ -23,13 +23,12 @@
import com.codahale.metrics.graphite.GraphiteSender;
import com.codahale.metrics.graphite.PickledGraphite;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.druid.java.util.common.ISE;
+import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.emitter.core.Emitter;
import io.druid.java.util.emitter.core.Event;
import io.druid.java.util.emitter.service.AlertEvent;
import io.druid.java.util.emitter.service.ServiceMetricEvent;
-
-import io.druid.java.util.common.ISE;
-import io.druid.java.util.common.logger.Logger;
import io.druid.server.log.EmittingRequestLogger;
import java.io.IOException;
@@ -218,7 +217,7 @@ public void run()
}
@Override
- public void flush() throws IOException
+ public void flush()
{
if (started.get()) {
Future future = exec.schedule(new ConsumerRunnable(), 0, TimeUnit.MILLISECONDS);
@@ -236,7 +235,7 @@ public void flush() throws IOException
}
@Override
- public void close() throws IOException
+ public void close()
{
flush();
started.set(false);
diff --git a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java
index 7cc7bdbcd739..4121ea606504 100644
--- a/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java
+++ b/extensions-contrib/graphite-emitter/src/main/java/io/druid/emitter/graphite/WhiteListBasedConverter.java
@@ -33,9 +33,9 @@
import com.google.common.collect.ImmutableSortedMap;
import com.google.common.io.Files;
import com.google.common.io.Resources;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
import java.io.File;
import java.io.IOException;
@@ -121,11 +121,6 @@ public boolean replaceSlashWithDot()
return replaceSlashWithDot;
}
- public ImmutableSortedMap> getWhiteListDimsMapper()
- {
- return whiteListDimsMapper;
- }
-
/**
* @param event Event subject to filtering
*
diff --git a/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java b/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java
index f4d139c2ec08..1b69e3547b57 100644
--- a/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java
+++ b/extensions-contrib/graphite-emitter/src/test/java/io/druid/emitter/graphite/WhiteListBasedConverterTest.java
@@ -20,9 +20,10 @@
package io.druid.emitter.graphite;
import com.google.common.collect.Maps;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
+import io.druid.annotations.UsedByJUnitParamsRunner;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
import junitparams.JUnitParamsRunner;
import junitparams.Parameters;
import org.apache.commons.io.IOUtils;
@@ -145,6 +146,7 @@ public void testWhiteListedStringArrayDimension() throws IOException
Assert.assertEquals(defaultNamespace + ".g1.jvm/gc/cpu", graphiteEvent.getEventPath());
}
+ @UsedByJUnitParamsRunner
private Object[] parametersForTestGetPath()
{
return new Object[]{
diff --git a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java
index c65fa808d70c..f593f2c5a27f 100644
--- a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java
+++ b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java
@@ -25,8 +25,6 @@
import com.google.common.collect.Iterators;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
-import io.druid.java.util.common.parsers.ParseException;
-import io.druid.java.util.emitter.EmittingLogger;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Committer;
import io.druid.data.input.FirehoseFactoryV2;
@@ -34,6 +32,8 @@
import io.druid.data.input.InputRow;
import io.druid.firehose.kafka.KafkaSimpleConsumer.BytesMessageWithOffset;
import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.parsers.ParseException;
+import io.druid.java.util.emitter.EmittingLogger;
import java.io.Closeable;
import java.io.IOException;
@@ -147,7 +147,7 @@ private Map loadOffsetFromPreviousMetaData(Object lastCommit)
}
@Override
- public FirehoseV2 connect(final ByteBufferInputRowParser firehoseParser, Object lastCommit) throws IOException
+ public FirehoseV2 connect(final ByteBufferInputRowParser firehoseParser, Object lastCommit)
{
final Map lastOffsets = loadOffsetFromPreviousMetaData(lastCommit);
@@ -185,7 +185,7 @@ public FirehoseV2 connect(final ByteBufferInputRowParser firehoseParser, Object
}
@Override
- public void start() throws Exception
+ public void start()
{
}
@@ -336,7 +336,7 @@ public void run()
}
@Override
- public synchronized void close() throws IOException
+ public synchronized void close()
{
if (stopped.compareAndSet(false, true)) {
thread.interrupt();
diff --git a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java
index 30060f60ec7d..92c000a4d55f 100644
--- a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java
+++ b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/KafkaEmitter.java
@@ -22,15 +22,15 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
-import io.druid.java.util.emitter.core.Emitter;
-import io.druid.java.util.emitter.core.Event;
-import io.druid.java.util.emitter.service.AlertEvent;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
import io.druid.emitter.kafka.MemoryBoundLinkedBlockingQueue.ObjectContainer;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+import io.druid.java.util.emitter.service.AlertEvent;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
@@ -38,7 +38,6 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;
-import java.io.IOException;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.Executors;
@@ -188,14 +187,14 @@ public void emit(final Event event)
}
@Override
- public void flush() throws IOException
+ public void flush()
{
producer.flush();
}
@Override
@LifecycleStop
- public void close() throws IOException
+ public void close()
{
scheduler.shutdownNow();
producer.close();
diff --git a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java
index b12d2a75d16e..25f199111b3a 100644
--- a/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java
+++ b/extensions-contrib/kafka-emitter/src/main/java/io/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java
@@ -61,16 +61,6 @@ public ObjectContainer take() throws InterruptedException
return ret;
}
- public long getAvailableBuffer()
- {
- return memoryBound - currentMemory.get();
- }
-
- public int size()
- {
- return queue.size();
- }
-
public static class ObjectContainer
{
private T data;
diff --git a/extensions-contrib/kafka-emitter/src/test/java/io/druid/emitter/kafka/KafkaEmitterConfigTest.java b/extensions-contrib/kafka-emitter/src/test/java/io/druid/emitter/kafka/KafkaEmitterConfigTest.java
index 3b91706fbf02..7526de287599 100644
--- a/extensions-contrib/kafka-emitter/src/test/java/io/druid/emitter/kafka/KafkaEmitterConfigTest.java
+++ b/extensions-contrib/kafka-emitter/src/test/java/io/druid/emitter/kafka/KafkaEmitterConfigTest.java
@@ -54,13 +54,14 @@ public void testSerDeserKafkaEmitterConfig() throws IOException
}
@Test
- public void testSerDeNotRequiredKafkaProducerConfig() throws IOException
+ public void testSerDeNotRequiredKafkaProducerConfig()
{
KafkaEmitterConfig kafkaEmitterConfig = new KafkaEmitterConfig("localhost:9092", "metricTest",
"alertTest", "clusterNameTest",
null
);
try {
+ @SuppressWarnings("unused")
KafkaEmitter emitter = new KafkaEmitter(kafkaEmitterConfig, mapper);
}
catch (NullPointerException e) {
diff --git a/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/OpentsdbEmitter.java b/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/OpentsdbEmitter.java
index dd5ab7ae849d..8f22adee41d0 100644
--- a/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/OpentsdbEmitter.java
+++ b/extensions-contrib/opentsdb-emitter/src/main/java/io/druid/emitter/opentsdb/OpentsdbEmitter.java
@@ -25,8 +25,6 @@
import io.druid.java.util.emitter.core.Event;
import io.druid.java.util.emitter.service.ServiceMetricEvent;
-import java.io.IOException;
-
public class OpentsdbEmitter implements Emitter
{
private static final Logger log = new Logger(OpentsdbEmitter.class);
@@ -69,13 +67,13 @@ public void emit(Event event)
}
@Override
- public void flush() throws IOException
+ public void flush()
{
sender.flush();
}
@Override
- public void close() throws IOException
+ public void close()
{
sender.close();
}
diff --git a/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/EventConverterTest.java b/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/EventConverterTest.java
index 90d8ac745e46..0afc72c9170e 100644
--- a/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/EventConverterTest.java
+++ b/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/EventConverterTest.java
@@ -20,8 +20,8 @@
package io.druid.emitter.opentsdb;
import com.fasterxml.jackson.databind.ObjectMapper;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
@@ -50,7 +50,7 @@ public void testSanitize()
}
@Test
- public void testConvert() throws Exception
+ public void testConvert()
{
DateTime dateTime = DateTimes.nowUtc();
ServiceMetricEvent configuredEvent = new ServiceMetricEvent.Builder()
diff --git a/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/OpentsdbSenderTest.java b/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/OpentsdbSenderTest.java
index 84cd167e1213..6b2d52e8a7c0 100644
--- a/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/OpentsdbSenderTest.java
+++ b/extensions-contrib/opentsdb-emitter/src/test/java/io/druid/emitter/opentsdb/OpentsdbSenderTest.java
@@ -25,7 +25,7 @@
public class OpentsdbSenderTest
{
@Test
- public void testUrl() throws Exception
+ public void testUrl()
{
OpentsdbSender sender = new OpentsdbSender("localhost", 9999, 2000, 2000, 100, 1000);
String expectedUrl = "http://localhost:9999/api/put";
diff --git a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java
index f6759da27e90..d348f5e6eceb 100644
--- a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java
+++ b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java
@@ -314,7 +314,7 @@ public void handleShutdownSignal(String consumerTag, ShutdownSignalException sig
}
@Override
- public void handleCancel(String consumerTag) throws IOException
+ public void handleCancel(String consumerTag)
{
_queue.clear();
}
@@ -326,7 +326,6 @@ public void handleDelivery(
AMQP.BasicProperties properties,
byte[] body
)
- throws IOException
{
this._queue.add(new Delivery(envelope, properties, body));
}
diff --git a/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java b/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java
index 2817bb013aa6..8ebb7d551242 100644
--- a/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java
+++ b/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java
@@ -63,7 +63,7 @@ public long getExpiration()
};
@Before
- public void setUp() throws Exception
+ public void setUp()
{
JedisPoolConfig poolConfig = new JedisPoolConfig();
poolConfig.setMaxTotal(cacheConfig.getMaxTotalConnections());
@@ -135,7 +135,7 @@ public void testSimpleInjection()
}
@Test
- public void testSanity() throws Exception
+ public void testSanity()
{
Assert.assertNull(cache.get(new Cache.NamedKey("a", HI)));
put(cache, "a", HI, 0);
@@ -158,7 +158,7 @@ public void testSanity() throws Exception
}
@Test
- public void testGetBulk() throws Exception
+ public void testGetBulk()
{
Assert.assertNull(cache.get(new Cache.NamedKey("the", HI)));
diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerConnector.java
index 2e96734225e7..ca3ae4ca4a70 100644
--- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerConnector.java
+++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/io/druid/metadata/storage/sqlserver/SQLServerConnector.java
@@ -224,13 +224,13 @@ public Void insertOrUpdate(
final String keyColumn,
final String valueColumn,
final String key,
- final byte[] value) throws Exception
+ final byte[] value)
{
return getDBI().withHandle(
new HandleCallback()
{
@Override
- public Void withHandle(Handle handle) throws Exception
+ public Void withHandle(Handle handle)
{
handle.createStatement(StringUtils.format(
"MERGE INTO %1$s WITH (UPDLOCK, HOLDLOCK) as target"
diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/io/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/io/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java
index 7c90fd8440b4..8ba707c41f30 100644
--- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/io/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java
+++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/io/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java
@@ -18,22 +18,20 @@
*/
package io.druid.metadata.storage.sqlserver;
-import java.sql.SQLException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
import com.google.common.base.Suppliers;
-
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.sql.SQLException;
@SuppressWarnings("nls")
public class SQLServerConnectorTest
{
@Test
- public void testIsTransientException() throws Exception
+ public void testIsTransientException()
{
SQLServerConnector connector = new SQLServerConnector(
Suppliers.ofInstance(new MetadataStorageConnectorConfig()),
diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java
index a24d3eef11ce..b53798e08534 100644
--- a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java
+++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java
@@ -22,15 +22,14 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
-import io.druid.java.util.emitter.core.Emitter;
-import io.druid.java.util.emitter.core.Event;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
import com.timgroup.statsd.NonBlockingStatsDClient;
import com.timgroup.statsd.StatsDClient;
import com.timgroup.statsd.StatsDClientErrorHandler;
import io.druid.java.util.common.logger.Logger;
+import io.druid.java.util.emitter.core.Emitter;
+import io.druid.java.util.emitter.core.Event;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
-import java.io.IOException;
import java.util.Map;
/**
@@ -127,10 +126,10 @@ public void emit(Event event)
}
@Override
- public void flush() throws IOException {}
+ public void flush() {}
@Override
- public void close() throws IOException
+ public void close()
{
statsd.stop();
}
diff --git a/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java b/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java
index 62132a266cf1..02e563719683 100644
--- a/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java
+++ b/extensions-contrib/statsd-emitter/src/test/java/DimensionConverterTest.java
@@ -19,10 +19,10 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
-import io.druid.java.util.emitter.service.ServiceMetricEvent;
import io.druid.emitter.statsd.DimensionConverter;
import io.druid.emitter.statsd.StatsDMetric;
import io.druid.java.util.common.DateTimes;
+import io.druid.java.util.emitter.service.ServiceMetricEvent;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
@@ -32,7 +32,7 @@
public class DimensionConverterTest
{
@Test
- public void testConvert() throws Exception
+ public void testConvert()
{
DimensionConverter dimensionConverter = new DimensionConverter(new ObjectMapper(), null);
ServiceMetricEvent event = new ServiceMetricEvent.Builder()
diff --git a/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java b/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java
index ed6a794c640b..ac20c849d08a 100644
--- a/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java
+++ b/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java
@@ -35,7 +35,6 @@
import io.druid.js.JavaScriptConfig;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.io.BytesWritable;
-import org.apache.thrift.TException;
import org.apache.thrift.TSerializer;
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.protocol.TCompactProtocol;
@@ -59,7 +58,7 @@ public class ThriftInputRowParserTest
private ParseSpec parseSpec;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
parseSpec = new JSONParseSpec(new TimestampSpec("date", "auto", null),
new DimensionsSpec(Lists.newArrayList(
@@ -149,7 +148,7 @@ public void testDisableJavaScript()
parser.parseBatch(ByteBuffer.allocate(1)).get(0);
}
- public void serializationAndTest(ThriftInputRowParser parser, byte[] bytes) throws TException
+ public void serializationAndTest(ThriftInputRowParser parser, byte[] bytes)
{
ByteBuffer buffer = ByteBuffer.wrap(bytes);
diff --git a/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java b/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java
index 263945568e4c..58fbdc700622 100644
--- a/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java
+++ b/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampAggregationSelectTest.java
@@ -90,7 +90,7 @@ public TimestampAggregationSelectTest(String aggType, String aggField, Long expe
}
@Before
- public void setup() throws Exception
+ public void setup()
{
helper = AggregationTestHelper.createSelectQueryAggregationTestHelper(
new TimestampMinMaxModule().getJacksonModules(),
diff --git a/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampGroupByAggregationTest.java b/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampGroupByAggregationTest.java
index 38112b8bfd8c..ebcc0b90635d 100644
--- a/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampGroupByAggregationTest.java
+++ b/extensions-contrib/time-min-max/src/test/java/io/druid/query/aggregation/TimestampGroupByAggregationTest.java
@@ -100,7 +100,7 @@ public TimestampGroupByAggregationTest(
}
@Before
- public void setup() throws Exception
+ public void setup()
{
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
new TimestampMinMaxModule().getJacksonModules(),
diff --git a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java
index 3386857d24cd..63155edd2ea8 100644
--- a/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java
+++ b/extensions-contrib/virtual-columns/src/test/java/io/druid/segment/MapVirtualColumnTest.java
@@ -141,7 +141,7 @@ private Druids.SelectQueryBuilder testBuilder()
}
@Test
- public void testBasic() throws Exception
+ public void testBasic()
{
Druids.SelectQueryBuilder builder = testBuilder();
@@ -185,7 +185,7 @@ private Map mapOf(Object... elements)
return map;
}
- private void checkSelectQuery(SelectQuery searchQuery, List expected) throws Exception
+ private void checkSelectQuery(SelectQuery searchQuery, List expected)
{
List> results = runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()).toList();
Assert.assertEquals(1, results.size());
diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java
index 452eb6d46534..4832c46804d8 100644
--- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java
+++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueInputFormat.java
@@ -34,6 +34,7 @@
import java.io.IOException;
+@SuppressWarnings("unused") // This class should be specified and used via hadoop ioConfig, see avro.md
public class AvroValueInputFormat extends FileInputFormat
{
private static final Logger log = new Logger(AvroValueInputFormat.class);
@@ -46,7 +47,7 @@ public class AvroValueInputFormat extends FileInputFormat createRecordReader(
InputSplit split, TaskAttemptContext context
- ) throws IOException, InterruptedException
+ ) throws IOException
{
Schema readerSchema = AvroJob.getInputValueSchema(context.getConfiguration());
diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueRecordReader.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueRecordReader.java
index 241f169a5767..5bef4562528b 100644
--- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueRecordReader.java
+++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/AvroValueRecordReader.java
@@ -23,8 +23,6 @@
import org.apache.avro.mapreduce.AvroRecordReaderBase;
import org.apache.hadoop.io.NullWritable;
-import java.io.IOException;
-
public class AvroValueRecordReader extends AvroRecordReaderBase
{
public AvroValueRecordReader(Schema readerSchema)
@@ -36,7 +34,7 @@ public AvroValueRecordReader(Schema readerSchema)
* {@inheritDoc}
*/
@Override
- public NullWritable getCurrentKey() throws IOException, InterruptedException
+ public NullWritable getCurrentKey()
{
return NullWritable.get();
}
@@ -45,7 +43,7 @@ public NullWritable getCurrentKey() throws IOException, InterruptedException
* {@inheritDoc}
*/
@Override
- public GenericRecord getCurrentValue() throws IOException, InterruptedException
+ public GenericRecord getCurrentValue()
{
return getCurrentRecord();
}
diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java
index 8f9e151feb2e..bbc4b101e71b 100644
--- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java
+++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java
@@ -52,8 +52,7 @@ public Pair getSubjectAndId(ByteBuffer payload)
return new Pair<>(topic, payload.getInt());
}
- @Override
- public void putSubjectAndId(String subject, Integer id, ByteBuffer payload)
+ public void putSubjectAndId(Integer id, ByteBuffer payload)
{
payload.putInt(id);
}
diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/SubjectAndIdConverter.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/SubjectAndIdConverter.java
index 40769a462a6b..61814acc845c 100644
--- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/SubjectAndIdConverter.java
+++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/schemarepo/SubjectAndIdConverter.java
@@ -51,8 +51,6 @@ public interface SubjectAndIdConverter
Pair getSubjectAndId(ByteBuffer payload);
- void putSubjectAndId(SUBJECT subject, ID id, ByteBuffer payload);
-
Converter getSubjectConverter();
Converter getIdConverter();
diff --git a/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java
index 0f6e3ce9a416..b5a3184c5bc6 100644
--- a/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java
+++ b/extensions-core/avro-extensions/src/test/java/io/druid/data/input/AvroStreamInputRowParserTest.java
@@ -31,12 +31,12 @@
import io.druid.data.input.avro.AvroParseSpec;
import io.druid.data.input.avro.SchemaRepoBasedAvroBytesDecoder;
import io.druid.data.input.impl.DimensionsSpec;
-import io.druid.java.util.common.parsers.JSONPathFieldSpec;
-import io.druid.java.util.common.parsers.JSONPathFieldType;
-import io.druid.java.util.common.parsers.JSONPathSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.data.input.schemarepo.Avro1124RESTRepositoryClientWrapper;
import io.druid.data.input.schemarepo.Avro1124SubjectAndIdConverter;
+import io.druid.java.util.common.parsers.JSONPathFieldSpec;
+import io.druid.java.util.common.parsers.JSONPathFieldType;
+import io.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter;
@@ -206,7 +206,7 @@ public void testParse() throws SchemaValidationException, IOException
);
Integer id = repositoryClient.registerSchema(TOPIC, SomeAvroDatum.getClassSchema());
ByteBuffer byteBuffer = ByteBuffer.allocate(4);
- converter.putSubjectAndId(TOPIC, id, byteBuffer);
+ converter.putSubjectAndId(id, byteBuffer);
ByteArrayOutputStream out = new ByteArrayOutputStream();
out.write(byteBuffer.array());
// encode data
@@ -247,7 +247,7 @@ public void testParseSchemaless() throws SchemaValidationException, IOException
);
Integer id = repositoryClient.registerSchema(TOPIC, SomeAvroDatum.getClassSchema());
ByteBuffer byteBuffer = ByteBuffer.allocate(4);
- converter.putSubjectAndId(TOPIC, id, byteBuffer);
+ converter.putSubjectAndId(id, byteBuffer);
ByteArrayOutputStream out = new ByteArrayOutputStream();
out.write(byteBuffer.array());
// encode data
@@ -321,7 +321,7 @@ public Integer apply(@Nullable String input)
assertEquals(SOME_INT_VALUE, inputRow.getMetric("someInt"));
}
- public static SomeAvroDatum buildSomeAvroDatum() throws IOException
+ public static SomeAvroDatum buildSomeAvroDatum()
{
return SomeAvroDatum.newBuilder()
.setTimestamp(DATE_TIME.getMillis())
diff --git a/extensions-core/avro-extensions/src/test/java/io/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java b/extensions-core/avro-extensions/src/test/java/io/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java
index 349726a0116a..cd3290b0d26e 100644
--- a/extensions-core/avro-extensions/src/test/java/io/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java
+++ b/extensions-core/avro-extensions/src/test/java/io/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoderTest.java
@@ -49,7 +49,7 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
SchemaRegistryClient registry;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
registry = mock(SchemaRegistryClient.class);
}
diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java
index fc24dbbe8301..260dc12b3507 100644
--- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java
+++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java
@@ -79,7 +79,7 @@ public SketchAggregationTest(final GroupByQueryConfig config)
}
@Parameterized.Parameters(name = "{0}")
- public static Collection> constructorFeeder() throws IOException
+ public static Collection> constructorFeeder()
{
final List constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
@@ -226,7 +226,7 @@ public void testSketchMergeAggregatorFactorySerde() throws Exception
}
@Test
- public void testSketchMergeFinalization() throws Exception
+ public void testSketchMergeFinalization()
{
SketchHolder sketch = SketchHolder.of(Sketches.updateSketchBuilder().setNominalEntries(128).build());
diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java
index 3bf0e13a8d68..870fa9794b0c 100644
--- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java
+++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationWithSimpleDataTest.java
@@ -71,7 +71,7 @@ public SketchAggregationWithSimpleDataTest(GroupByQueryConfig config)
}
@Parameterized.Parameters(name = "{0}")
- public static Collection> constructorFeeder() throws IOException
+ public static Collection> constructorFeeder()
{
final List constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java
index f01efca9560e..9ec602b5a600 100644
--- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java
+++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java
@@ -73,7 +73,7 @@ public OldApiSketchAggregationTest(final GroupByQueryConfig config)
}
@Parameterized.Parameters(name = "{0}")
- public static Collection> constructorFeeder() throws IOException
+ public static Collection> constructorFeeder()
{
final List constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/BasicAuthUtils.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/BasicAuthUtils.java
index 97d184b3d1f4..69f33edc36c4 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/BasicAuthUtils.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/BasicAuthUtils.java
@@ -80,11 +80,6 @@ public class BasicAuthUtils
{
};
- public static String getEncodedCredentials(final String unencodedCreds)
- {
- return Base64.getEncoder().encodeToString(StringUtils.toUtf8(unencodedCreds));
- }
-
public static byte[] hashPassword(final char[] password, final byte[] salt, final int iterations)
{
try {
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java
index b127687999bd..99ecbd7f63ff 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/BasicHTTPAuthenticator.java
@@ -144,7 +144,7 @@ public BasicAuthDBConfig getDbConfig()
public class BasicHTTPAuthenticationFilter implements Filter
{
@Override
- public void init(FilterConfig filterConfig) throws ServletException
+ public void init(FilterConfig filterConfig)
{
}
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java
index 5ab501b5166f..82d18f597c25 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java
@@ -21,8 +21,6 @@
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
-import io.druid.java.util.emitter.EmittingLogger;
-import io.druid.java.util.http.client.HttpClient;
import io.druid.concurrent.LifecycleLock;
import io.druid.discovery.DruidNodeDiscoveryProvider;
import io.druid.guice.ManageLifecycle;
@@ -30,6 +28,7 @@
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.http.client.HttpClient;
import io.druid.security.basic.BasicAuthDBConfig;
import io.druid.security.basic.CommonCacheNotifier;
import io.druid.security.basic.authentication.BasicHTTPAuthenticator;
@@ -43,7 +42,6 @@
@ManageLifecycle
public class CoordinatorBasicAuthenticatorCacheNotifier implements BasicAuthenticatorCacheNotifier
{
- private static final EmittingLogger LOG = new EmittingLogger(CoordinatorBasicAuthenticatorCacheNotifier.class);
private final LifecycleLock lifecycleLock = new LifecycleLock();
private CommonCacheNotifier cacheNotifier;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java
index 872cdc103208..ccc17d51f1ed 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java
@@ -22,7 +22,6 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
-import io.druid.java.util.emitter.EmittingLogger;
import io.druid.common.config.ConfigManager;
import io.druid.concurrent.LifecycleLock;
import io.druid.guice.ManageLifecycle;
@@ -33,6 +32,7 @@
import io.druid.java.util.common.concurrent.ScheduledExecutors;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.emitter.EmittingLogger;
import io.druid.metadata.MetadataCASUpdate;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageTablesConfig;
@@ -166,7 +166,7 @@ public void start()
new Callable()
{
@Override
- public ScheduledExecutors.Signal call() throws Exception
+ public ScheduledExecutors.Signal call()
{
if (stopped) {
return ScheduledExecutors.Signal.STOP;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java
index 9f7a1def62a3..9f4ee4ba2ad9 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java
@@ -21,8 +21,6 @@
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
-import io.druid.java.util.emitter.EmittingLogger;
-import io.druid.java.util.http.client.HttpClient;
import io.druid.concurrent.LifecycleLock;
import io.druid.discovery.DruidNodeDiscoveryProvider;
import io.druid.guice.ManageLifecycle;
@@ -30,6 +28,7 @@
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.http.client.HttpClient;
import io.druid.security.basic.BasicAuthDBConfig;
import io.druid.security.basic.CommonCacheNotifier;
import io.druid.security.basic.authorization.BasicRoleBasedAuthorizer;
@@ -43,7 +42,6 @@
@ManageLifecycle
public class CoordinatorBasicAuthorizerCacheNotifier implements BasicAuthorizerCacheNotifier
{
- private static final EmittingLogger LOG = new EmittingLogger(CoordinatorBasicAuthorizerCacheNotifier.class);
private final LifecycleLock lifecycleLock = new LifecycleLock();
private CommonCacheNotifier cacheNotifier;
diff --git a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java
index adc0a60591c5..551b6a6a7fd8 100644
--- a/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java
+++ b/extensions-core/druid-basic-security/src/main/java/io/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java
@@ -23,7 +23,6 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
-import io.druid.java.util.emitter.EmittingLogger;
import io.druid.common.config.ConfigManager;
import io.druid.concurrent.LifecycleLock;
import io.druid.guice.ManageLifecycle;
@@ -34,6 +33,7 @@
import io.druid.java.util.common.concurrent.ScheduledExecutors;
import io.druid.java.util.common.lifecycle.LifecycleStart;
import io.druid.java.util.common.lifecycle.LifecycleStop;
+import io.druid.java.util.emitter.EmittingLogger;
import io.druid.metadata.MetadataCASUpdate;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageTablesConfig;
@@ -140,7 +140,6 @@ public void start()
if (authorizer instanceof BasicRoleBasedAuthorizer) {
String authorizerName = entry.getKey();
authorizerNames.add(authorizerName);
- BasicRoleBasedAuthorizer basicRoleBasedAuthorizer = (BasicRoleBasedAuthorizer) authorizer;
byte[] userMapBytes = getCurrentUserMapBytes(authorizerName);
Map userMap = BasicAuthUtils.deserializeAuthorizerUserMap(
@@ -167,7 +166,7 @@ public void start()
new Callable()
{
@Override
- public ScheduledExecutors.Signal call() throws Exception
+ public ScheduledExecutors.Signal call()
{
if (stopped) {
return ScheduledExecutors.Signal.STOP;
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java
index b65a3647b9e8..e5742ad3a579 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java
@@ -21,30 +21,18 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import com.google.inject.Binder;
-import com.google.inject.Injector;
-import com.google.inject.Key;
-import com.google.inject.Module;
-import io.druid.guice.GuiceInjectors;
-import io.druid.guice.JsonConfigProvider;
-import io.druid.guice.annotations.Self;
-import io.druid.initialization.Initialization;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.metadata.TestDerbyConnector;
import io.druid.security.basic.BasicAuthCommonCacheConfig;
import io.druid.security.basic.BasicAuthUtils;
import io.druid.security.basic.BasicSecurityDBResourceException;
import io.druid.security.basic.authentication.BasicHTTPAuthenticator;
-import io.druid.security.basic.authentication.BasicHTTPEscalator;
import io.druid.security.basic.authentication.db.updater.CoordinatorBasicAuthenticatorMetadataStorageUpdater;
import io.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate;
import io.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials;
import io.druid.security.basic.authentication.entity.BasicAuthenticatorUser;
-import io.druid.server.DruidNode;
import io.druid.server.security.AuthenticatorMapper;
-import io.druid.server.security.Escalator;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -69,7 +57,7 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest
private ObjectMapper objectMapper;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
objectMapper = new ObjectMapper(new SmileFactory());
connector = derbyConnectorRule.getConnector();
@@ -104,7 +92,7 @@ public void setUp() throws Exception
}
@After
- public void tearDown() throws Exception
+ public void tearDown()
{
updater.stop();
}
@@ -167,46 +155,4 @@ public void setCredentials()
Assert.assertArrayEquals(credentials.getHash(), recalculatedHash);
}
- private Injector setupInjector()
- {
- return Initialization.makeInjectorWithModules(
- GuiceInjectors.makeStartupInjector(),
- ImmutableList.of(
- new Module()
- {
- @Override
- public void configure(Binder binder)
- {
- JsonConfigProvider.bindInstance(
- binder,
- Key.get(DruidNode.class, Self.class),
- new DruidNode("test", "localhost", null, null, true, false)
- );
-
- binder.bind(Escalator.class).toInstance(
- new BasicHTTPEscalator(null, null, null)
- );
-
- binder.bind(AuthenticatorMapper.class).toInstance(
- new AuthenticatorMapper(
- ImmutableMap.of(
- "test",
- new BasicHTTPAuthenticator(
- null,
- "test",
- "test",
- null,
- null,
- null,
- null,
- null
- )
- )
- )
- );
- }
- }
- )
- );
- }
}
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java
index 1bf3d54f2331..b14d04fa34a8 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java
@@ -21,32 +21,20 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
-import com.google.inject.Binder;
-import com.google.inject.Injector;
-import com.google.inject.Key;
-import com.google.inject.Module;
-import io.druid.guice.GuiceInjectors;
-import io.druid.guice.JsonConfigProvider;
-import io.druid.guice.annotations.Self;
-import io.druid.initialization.Initialization;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.metadata.TestDerbyConnector;
import io.druid.security.basic.BasicAuthCommonCacheConfig;
import io.druid.security.basic.BasicAuthUtils;
import io.druid.security.basic.authentication.BasicHTTPAuthenticator;
-import io.druid.security.basic.authentication.BasicHTTPEscalator;
import io.druid.security.basic.authentication.db.updater.CoordinatorBasicAuthenticatorMetadataStorageUpdater;
import io.druid.security.basic.authentication.endpoint.BasicAuthenticatorResource;
import io.druid.security.basic.authentication.endpoint.CoordinatorBasicAuthenticatorResourceHandler;
import io.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate;
import io.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials;
import io.druid.security.basic.authentication.entity.BasicAuthenticatorUser;
-import io.druid.server.DruidNode;
import io.druid.server.security.AuthenticatorMapper;
-import io.druid.server.security.Escalator;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Assert;
@@ -78,7 +66,7 @@ public class CoordinatorBasicAuthenticatorResourceTest
private HttpServletRequest req;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
req = EasyMock.createStrictMock(HttpServletRequest.class);
@@ -137,7 +125,7 @@ public void setUp() throws Exception
}
@After
- public void tearDown() throws Exception
+ public void tearDown()
{
storageUpdater.stop();
}
@@ -296,28 +284,4 @@ private static Map errorMapWithMsg(String errorMsg)
return ImmutableMap.of("error", errorMsg);
}
- private Injector setupInjector()
- {
- return Initialization.makeInjectorWithModules(
- GuiceInjectors.makeStartupInjector(),
- ImmutableList.of(
- new Module()
- {
- @Override
- public void configure(Binder binder)
- {
- JsonConfigProvider.bindInstance(
- binder,
- Key.get(DruidNode.class, Self.class),
- new DruidNode("test", "localhost", null, null, true, false)
- );
-
- binder.bind(Escalator.class).toInstance(
- new BasicHTTPEscalator(null, null, null)
- );
- }
- }
- )
- );
- }
}
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
index 6132753c684d..872c3766268a 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/BasicRoleBasedAuthorizerTest.java
@@ -57,7 +57,7 @@ public class BasicRoleBasedAuthorizerTest
private CoordinatorBasicAuthorizerMetadataStorageUpdater updater;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
connector = derbyConnectorRule.getConnector();
tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get();
@@ -96,7 +96,7 @@ public void setUp() throws Exception
}
@After
- public void tearDown() throws Exception
+ public void tearDown()
{
}
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java
index 76be80c694f8..5e0e38c92834 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java
@@ -86,7 +86,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest
private ObjectMapper objectMapper;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
objectMapper = new ObjectMapper(new SmileFactory());
connector = derbyConnectorRule.getConnector();
@@ -118,7 +118,7 @@ public void setUp() throws Exception
// user tests
@Test
- public void testCreateDeleteUser() throws Exception
+ public void testCreateDeleteUser()
{
updater.createUser(AUTHORIZER_NAME, "druid");
Map expectedUserMap = Maps.newHashMap(BASE_USER_MAP);
@@ -139,7 +139,7 @@ public void testCreateDeleteUser() throws Exception
}
@Test
- public void testDeleteNonExistentUser() throws Exception
+ public void testDeleteNonExistentUser()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("User [druid] does not exist.");
@@ -147,7 +147,7 @@ public void testDeleteNonExistentUser() throws Exception
}
@Test
- public void testCreateDuplicateUser() throws Exception
+ public void testCreateDuplicateUser()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("User [druid] already exists.");
@@ -157,7 +157,7 @@ public void testCreateDuplicateUser() throws Exception
// role tests
@Test
- public void testCreateDeleteRole() throws Exception
+ public void testCreateDeleteRole()
{
updater.createRole(AUTHORIZER_NAME, "druid");
Map expectedRoleMap = Maps.newHashMap(BASE_ROLE_MAP);
@@ -178,7 +178,7 @@ public void testCreateDeleteRole() throws Exception
}
@Test
- public void testDeleteNonExistentRole() throws Exception
+ public void testDeleteNonExistentRole()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("Role [druid] does not exist.");
@@ -186,7 +186,7 @@ public void testDeleteNonExistentRole() throws Exception
}
@Test
- public void testCreateDuplicateRole() throws Exception
+ public void testCreateDuplicateRole()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("Role [druid] already exists.");
@@ -196,7 +196,7 @@ public void testCreateDuplicateRole() throws Exception
// role and user tests
@Test
- public void testAddAndRemoveRole() throws Exception
+ public void testAddAndRemoveRole()
{
updater.createUser(AUTHORIZER_NAME, "druid");
updater.createRole(AUTHORIZER_NAME, "druidRole");
@@ -233,7 +233,7 @@ public void testAddAndRemoveRole() throws Exception
}
@Test
- public void testAddRoleToNonExistentUser() throws Exception
+ public void testAddRoleToNonExistentUser()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("User [nonUser] does not exist.");
@@ -242,7 +242,7 @@ public void testAddRoleToNonExistentUser() throws Exception
}
@Test
- public void testAddNonexistentRoleToUser() throws Exception
+ public void testAddNonexistentRoleToUser()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("Role [nonRole] does not exist.");
@@ -251,7 +251,7 @@ public void testAddNonexistentRoleToUser() throws Exception
}
@Test
- public void testAddExistingRoleToUserFails() throws Exception
+ public void testAddExistingRoleToUserFails()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("User [druid] already has role [druidRole].");
@@ -262,7 +262,7 @@ public void testAddExistingRoleToUserFails() throws Exception
}
@Test
- public void testUnassignInvalidRoleAssignmentFails() throws Exception
+ public void testUnassignInvalidRoleAssignmentFails()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("User [druid] does not have role [druidRole].");
@@ -294,7 +294,7 @@ public void testUnassignInvalidRoleAssignmentFails() throws Exception
// role and permission tests
@Test
- public void testSetRolePermissions() throws Exception
+ public void testSetRolePermissions()
{
updater.createUser(AUTHORIZER_NAME, "druid");
updater.createRole(AUTHORIZER_NAME, "druidRole");
@@ -343,7 +343,7 @@ public void testSetRolePermissions() throws Exception
}
@Test
- public void testAddPermissionToNonExistentRole() throws Exception
+ public void testAddPermissionToNonExistentRole()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("Role [druidRole] does not exist.");
@@ -359,7 +359,7 @@ public void testAddPermissionToNonExistentRole() throws Exception
}
@Test
- public void testAddBadPermission() throws Exception
+ public void testAddBadPermission()
{
expectedException.expect(BasicSecurityDBResourceException.class);
expectedException.expectMessage("Invalid permission, resource name regex[??????????] does not compile.");
diff --git a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java
index 1a2f42103d19..660c2e6d6c06 100644
--- a/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java
+++ b/extensions-core/druid-basic-security/src/test/java/io/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java
@@ -75,7 +75,7 @@ public class CoordinatorBasicAuthorizerResourceTest
private HttpServletRequest req;
@Before
- public void setUp() throws Exception
+ public void setUp()
{
req = EasyMock.createStrictMock(HttpServletRequest.class);
@@ -124,7 +124,7 @@ public void setUp() throws Exception
}
@After
- public void tearDown() throws Exception
+ public void tearDown()
{
storageUpdater.stop();
}
@@ -289,7 +289,7 @@ public void testCreateDeleteRole()
}
@Test
- public void testRoleAssignment() throws Exception
+ public void testRoleAssignment()
{
Response response = resource.createRole(req, AUTHORIZER_NAME, "druidRole");
Assert.assertEquals(200, response.getStatus());
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java
index 4b58e18180b0..341bf911157c 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java
@@ -125,7 +125,7 @@ public void init(Properties config) throws ServletException
{
@Override
- public GSSManager run() throws Exception
+ public GSSManager run()
{
return GSSManager.getInstance();
}
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java
index f7abbbbdb573..8f8317c5cbab 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java
@@ -92,7 +92,6 @@ public static String kerberosChallenge(String server) throws AuthenticationExcep
}
public static void authenticateIfRequired(String internalClientPrincipal, String internalClientKeytab)
- throws IOException
{
if (!Strings.isNullOrEmpty(internalClientPrincipal) && !Strings.isNullOrEmpty(internalClientKeytab)) {
Configuration conf = new Configuration();
diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java
index 7e58b178574d..fbdfb066c8e4 100644
--- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java
+++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java
@@ -160,7 +160,7 @@ public void init(FilterConfig filterConfig) throws ServletException
SignerSecretProvider signerSecretProvider = new SignerSecretProvider()
{
@Override
- public void init(Properties config, ServletContext servletContext, long tokenValidity) throws Exception
+ public void init(Properties config, ServletContext servletContext, long tokenValidity)
{
}
@@ -186,7 +186,7 @@ public byte[][] getAllSecrets()
// Copied from hadoop-auth's AuthenticationFilter, to allow us to change error response handling in doFilterSuper
@Override
- protected AuthenticationToken getToken(HttpServletRequest request) throws IOException, AuthenticationException
+ protected AuthenticationToken getToken(HttpServletRequest request) throws AuthenticationException
{
AuthenticationToken token = null;
String tokenStr = null;
diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java
index 66a7c4cc334f..81ce3cf4a4ef 100644
--- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java
+++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java
@@ -31,10 +31,8 @@
import io.druid.java.util.common.UOE;
import io.druid.java.util.common.io.NativeIO;
import io.druid.java.util.common.logger.Logger;
-import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.segment.loading.URIDataPuller;
-import io.druid.timeline.DataSegment;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@@ -53,7 +51,7 @@
/**
*/
-public class HdfsDataSegmentPuller implements DataSegmentPuller, URIDataPuller
+public class HdfsDataSegmentPuller implements URIDataPuller
{
public static final int DEFAULT_RETRY_COUNT = 3;
@@ -115,19 +113,19 @@ public OutputStream openOutputStream() throws IOException
}
@Override
- public Reader openReader(boolean ignoreEncodingErrors) throws IOException
+ public Reader openReader(boolean ignoreEncodingErrors)
{
throw new UOE("HDFS Reader not supported");
}
@Override
- public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException
+ public CharSequence getCharContent(boolean ignoreEncodingErrors)
{
throw new UOE("HDFS CharSequence not supported");
}
@Override
- public Writer openWriter() throws IOException
+ public Writer openWriter()
{
throw new UOE("HDFS Writer not supported");
}
@@ -167,15 +165,14 @@ public HdfsDataSegmentPuller(final Configuration config)
this.config = config;
}
-
- @Override
- public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException
- {
- getSegmentFiles(getPath(segment), dir);
- }
-
- public FileUtils.FileCopyResult getSegmentFiles(final Path path, final File outDir) throws SegmentLoadingException
+ FileUtils.FileCopyResult getSegmentFiles(final Path path, final File outDir) throws SegmentLoadingException
{
+ try {
+ org.apache.commons.io.FileUtils.forceMkdir(outDir);
+ }
+ catch (IOException e) {
+ throw new SegmentLoadingException(e, "");
+ }
try {
final FileSystem fs = path.getFileSystem(config);
if (fs.isDirectory(path)) {
@@ -277,14 +274,6 @@ public InputStream openStream() throws IOException
}
}
- public FileUtils.FileCopyResult getSegmentFiles(URI uri, File outDir) throws SegmentLoadingException
- {
- if (!uri.getScheme().equalsIgnoreCase(HdfsStorageDruidModule.SCHEME)) {
- throw new SegmentLoadingException("Don't know how to load SCHEME for URI [%s]", uri.toString());
- }
- return getSegmentFiles(new Path(uri), outDir);
- }
-
public InputStream getInputStream(Path path) throws IOException
{
return buildFileObject(path.toUri(), config).openInputStream();
@@ -340,9 +329,4 @@ public boolean apply(Throwable input)
}
};
}
-
- private Path getPath(DataSegment segment)
- {
- return new Path(String.valueOf(segment.getLoadSpec().get("path")));
- }
}
diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java
index 3374d4333329..96db0b28210d 100644
--- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java
+++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java
@@ -51,7 +51,6 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
{
private static final Logger log = new Logger(HdfsDataSegmentPusher.class);
- private final HdfsDataSegmentPusherConfig config;
private final Configuration hadoopConfig;
private final ObjectMapper jsonMapper;
private final String fullyQualifiedStorageDirectory;
@@ -63,7 +62,6 @@ public HdfsDataSegmentPusher(
ObjectMapper jsonMapper
) throws IOException
{
- this.config = config;
this.hadoopConfig = hadoopConfig;
this.jsonMapper = jsonMapper;
Path storageDir = new Path(config.getStorageDirectory());
diff --git a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java
index 19c40245c417..cc411c77d0fa 100644
--- a/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java
+++ b/extensions-core/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java
@@ -26,7 +26,6 @@
import com.google.inject.Binder;
import com.google.inject.Inject;
import com.google.inject.multibindings.MapBinder;
-
import io.druid.data.SearchableVersionedDataFinder;
import io.druid.guice.Binders;
import io.druid.guice.JsonConfigProvider;
@@ -34,7 +33,6 @@
import io.druid.guice.LifecycleModule;
import io.druid.guice.ManageLifecycle;
import io.druid.initialization.DruidModule;
-import io.druid.java.util.common.logger.Logger;
import io.druid.storage.hdfs.tasklog.HdfsTaskLogs;
import io.druid.storage.hdfs.tasklog.HdfsTaskLogsConfig;
import org.apache.hadoop.conf.Configuration;
@@ -48,7 +46,6 @@
*/
public class HdfsStorageDruidModule implements DruidModule
{
- private static final Logger log = new Logger(HdfsStorageDruidModule.class);
public static final String SCHEME = "hdfs";
private Properties props = null;
@@ -93,7 +90,6 @@ public void configure(Binder binder)
.to(HdfsFileTimestampVersionFinder.class)
.in(LazySingleton.class);
- Binders.dataSegmentPullerBinder(binder).addBinding(SCHEME).to(HdfsDataSegmentPuller.class).in(LazySingleton.class);
Binders.dataSegmentPusherBinder(binder).addBinding(SCHEME).to(HdfsDataSegmentPusher.class).in(LazySingleton.class);
Binders.dataSegmentKillerBinder(binder).addBinding(SCHEME).to(HdfsDataSegmentKiller.class).in(LazySingleton.class);
Binders.dataSegmentFinderBinder(binder).addBinding(SCHEME).to(HdfsDataSegmentFinder.class).in(LazySingleton.class);
diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java b/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java
index eb0b8c6a62e2..3a87bc5f898e 100644
--- a/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java
+++ b/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java
@@ -50,7 +50,7 @@ private HadoopFsWrapper() {}
*
* @throws IOException if trying to overwrite a non-empty directory
*/
- public static boolean rename(FileSystem fs, Path from, Path to, boolean replaceExisting) throws IOException
+ public static boolean rename(FileSystem fs, Path from, Path to, boolean replaceExisting)
{
try {
// Note: Using reflection instead of simpler
diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java
index 2e5b13dcf577..08d446b54654 100644
--- a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsFileTimestampVersionFinderTest.java
@@ -39,7 +39,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.net.URI;
import java.nio.file.Files;
import java.util.regex.Pattern;
@@ -48,7 +47,6 @@ public class HdfsFileTimestampVersionFinderTest
private static MiniDFSCluster miniCluster;
private static File hdfsTmpDir;
- private static URI uriBase;
private static Path filePath = new Path("/tmp/foo");
private static Path perTestPath = new Path("/tmp/tmp2");
private static String pathContents = "Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in culpa qui officia deserunt mollit anim id est laborum";
@@ -56,7 +54,7 @@ public class HdfsFileTimestampVersionFinderTest
private static Configuration conf;
@BeforeClass
- public static void setupStatic() throws IOException, ClassNotFoundException
+ public static void setupStatic() throws IOException
{
hdfsTmpDir = File.createTempFile("hdfsHandlerTest", "dir");
if (!hdfsTmpDir.delete()) {
@@ -65,7 +63,6 @@ public static void setupStatic() throws IOException, ClassNotFoundException
conf = new Configuration(true);
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, hdfsTmpDir.getAbsolutePath());
miniCluster = new MiniDFSCluster.Builder(conf).build();
- uriBase = miniCluster.getURI(0);
final File tmpFile = File.createTempFile("hdfsHandlerTest", ".data");
tmpFile.delete();
@@ -150,7 +147,7 @@ public void testAlreadyLatestVersion() throws IOException, InterruptedException
}
@Test
- public void testNoLatestVersion() throws IOException, InterruptedException
+ public void testNoLatestVersion() throws IOException
{
final Path oldPath = new Path(perTestPath, "555test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath));
diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPullerTest.java
similarity index 95%
rename from extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java
rename to extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPullerTest.java
index 343454cc35b2..6648b9e120b5 100644
--- a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentPullerTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPullerTest.java
@@ -17,13 +17,13 @@
* under the License.
*/
-package io.druid.segment.loading;
+package io.druid.storage.hdfs;
import com.google.common.io.ByteStreams;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
-import io.druid.storage.hdfs.HdfsDataSegmentPuller;
+import io.druid.segment.loading.SegmentLoadingException;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -60,7 +60,7 @@ public class HdfsDataSegmentPullerTest
private static Configuration conf;
@BeforeClass
- public static void setupStatic() throws IOException, ClassNotFoundException
+ public static void setupStatic() throws IOException
{
hdfsTmpDir = File.createTempFile("hdfsHandlerTest", "dir");
if (!hdfsTmpDir.delete()) {
@@ -133,7 +133,7 @@ public void testZip() throws IOException, SegmentLoadingException
}
try {
Assert.assertFalse(outFile.exists());
- puller.getSegmentFiles(uri, outTmpDir);
+ puller.getSegmentFiles(new Path(uri), outTmpDir);
Assert.assertTrue(outFile.exists());
Assert.assertArrayEquals(pathByteContents, Files.readAllBytes(outFile.toPath()));
@@ -172,7 +172,7 @@ public void testGZ() throws IOException, SegmentLoadingException
}
try {
Assert.assertFalse(outFile.exists());
- puller.getSegmentFiles(uri, outTmpDir);
+ puller.getSegmentFiles(new Path(uri), outTmpDir);
Assert.assertTrue(outFile.exists());
Assert.assertArrayEquals(pathByteContents, Files.readAllBytes(outFile.toPath()));
@@ -205,7 +205,7 @@ public void testDir() throws IOException, SegmentLoadingException
}
try {
Assert.assertFalse(outFile.exists());
- puller.getSegmentFiles(uri, outTmpDir);
+ puller.getSegmentFiles(new Path(uri), outTmpDir);
Assert.assertTrue(outFile.exists());
Assert.assertArrayEquals(pathByteContents, Files.readAllBytes(outFile.toPath()));
diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java
index bfc119f6498a..e53d1ea3e0a4 100644
--- a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java
+++ b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java
@@ -353,7 +353,7 @@ public Interval deserialize(JsonParser jsonParser, DeserializationContext deseri
}
@Test
- public void shouldNotHaveColonsInHdfsStorageDir() throws Exception
+ public void shouldNotHaveColonsInHdfsStorageDir()
{
Interval interval = Intervals.of("2011-10-01/2011-10-02");
diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java
index 1d9d13bc7fb0..6ed641d79c98 100644
--- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java
+++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java
@@ -42,9 +42,6 @@ static ApproximateHistogram combineHistograms(Object lhs, Object rhs)
}
private final BaseFloatColumnValueSelector selector;
- private final int resolution;
- private final float lowerLimit;
- private final float upperLimit;
private ApproximateHistogram histogram;
@@ -56,9 +53,6 @@ public ApproximateHistogramAggregator(
)
{
this.selector = selector;
- this.resolution = resolution;
- this.lowerLimit = lowerLimit;
- this.upperLimit = upperLimit;
this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
}
diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java
index 2ebd8b2bced8..057aeaa5e5e8 100644
--- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java
+++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java
@@ -26,9 +26,6 @@
public class ApproximateHistogramFoldingAggregator implements Aggregator
{
private final BaseObjectColumnValueSelector selector;
- private final int resolution;
- private final float lowerLimit;
- private final float upperLimit;
private ApproximateHistogram histogram;
private float[] tmpBufferP;
@@ -42,9 +39,6 @@ public ApproximateHistogramFoldingAggregator(
)
{
this.selector = selector;
- this.resolution = resolution;
- this.lowerLimit = lowerLimit;
- this.upperLimit = upperLimit;
this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit);
tmpBufferP = new float[resolution];
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java
index a34c35cef45e..ae5904c8cc82 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java
@@ -33,7 +33,6 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import java.io.IOException;
import java.util.Collection;
import java.util.List;
@@ -59,7 +58,7 @@ public ApproximateHistogramAggregationTest(final GroupByQueryConfig config)
}
@Parameterized.Parameters(name = "{0}")
- public static Collection> constructorFeeder() throws IOException
+ public static Collection> constructorFeeder()
{
final List constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java
index bb93c0130880..f405fc0a5ff8 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java
@@ -35,7 +35,7 @@ private void aggregateBuffer(TestFloatColumnSelector selector, BufferAggregator
}
@Test
- public void testBufferAggregate() throws Exception
+ public void testBufferAggregate()
{
final float[] values = {23, 19, 10, 16, 36, 2, 9, 32, 30, 45};
final int resolution = 5;
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java
index 7e7cc2e855e0..f338df289dbf 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java
@@ -26,6 +26,9 @@
import java.util.Locale;
import java.util.Random;
+/**
+ * TODO retwrite using JMH and move to the benchmarks module
+ */
public class ApproximateHistogramErrorBenchmark
{
private boolean debug = true;
@@ -43,18 +46,6 @@ public ApproximateHistogramErrorBenchmark setDebug(boolean debug)
return this;
}
- public ApproximateHistogramErrorBenchmark setNumBuckets(int numBuckets)
- {
- this.numBuckets = numBuckets;
- return this;
- }
-
- public ApproximateHistogramErrorBenchmark setNumBreaks(int numBreaks)
- {
- this.numBreaks = numBreaks;
- return this;
- }
-
public ApproximateHistogramErrorBenchmark setNumPerHist(int numPerHist)
{
this.numPerHist = numPerHist;
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
index 042a8c0f5027..13b3627a11d1 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java
@@ -40,7 +40,6 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -52,10 +51,9 @@ public class ApproximateHistogramGroupByQueryTest
{
private final QueryRunner runner;
private GroupByQueryRunnerFactory factory;
- private String testName;
@Parameterized.Parameters(name = "{0}")
- public static Iterable constructorFeeder() throws IOException
+ public static Iterable constructorFeeder()
{
final GroupByQueryConfig v1Config = new GroupByQueryConfig()
{
@@ -133,7 +131,6 @@ public String toString()
public ApproximateHistogramGroupByQueryTest(String testName, GroupByQueryRunnerFactory factory, QueryRunner runner)
{
- this.testName = testName;
this.factory = factory;
this.runner = runner;
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java
index 31d881d39616..07ff1f3d576c 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java
@@ -74,7 +74,7 @@ protected ApproximateHistogram buildHistogram(int size, float[] values, float lo
}
@Test
- public void testOffer() throws Exception
+ public void testOffer()
{
ApproximateHistogram h = buildHistogram(5, VALUES);
@@ -185,7 +185,7 @@ public void testFold()
}
@Test
- public void testFoldNothing() throws Exception
+ public void testFoldNothing()
{
ApproximateHistogram h1 = new ApproximateHistogram(10);
ApproximateHistogram h2 = new ApproximateHistogram(10);
@@ -195,7 +195,7 @@ public void testFoldNothing() throws Exception
}
@Test
- public void testFoldNothing2() throws Exception
+ public void testFoldNothing2()
{
ApproximateHistogram h1 = new ApproximateHistogram(10);
ApproximateHistogram h1Fast = new ApproximateHistogram(10);
@@ -221,6 +221,7 @@ public void testFoldNothing2() throws Exception
}
//@Test
+ @SuppressWarnings("unused") //TODO rewrite using JMH and move to the benchmarks module
public void testFoldSpeed()
{
final int combinedHistSize = 200;
@@ -251,6 +252,7 @@ public void testFoldSpeed()
float[] mergeBufferP = new float[combinedHistSize * 2];
long[] mergeBufferB = new long[combinedHistSize * 2];
+ @SuppressWarnings("unused") // make two different benchmarks - for fold, and foldFast (see commented line below)
float[] mergeBufferD = new float[combinedHistSize * 2];
for (int i = 0; i < count; ++i) {
diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
index d06d673b3945..787f2860c033 100644
--- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
+++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java
@@ -44,7 +44,6 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
@@ -56,7 +55,7 @@
public class ApproximateHistogramTopNQueryTest
{
@Parameterized.Parameters(name = "{0}")
- public static Iterable constructorFeeder() throws IOException
+ public static Iterable constructorFeeder()
{
return QueryRunnerTestHelper.transformToConstructionFeeder(
Iterables.concat(
diff --git a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java
index 9d7c6c5ffbe7..205d8b27c644 100644
--- a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java
+++ b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java
@@ -39,7 +39,6 @@
import javax.annotation.Nullable;
import java.io.File;
-import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Iterator;
import java.util.List;
@@ -71,7 +70,7 @@ public KafkaEightFirehoseFactory(
}
@Override
- public Firehose connect(final InputRowParser firehoseParser, File temporaryDirectory) throws IOException
+ public Firehose connect(final InputRowParser firehoseParser, File temporaryDirectory)
{
Set newDimExclus = Sets.union(
firehoseParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(),
@@ -165,7 +164,7 @@ public void run()
}
@Override
- public void close() throws IOException
+ public void close()
{
connector.shutdown();
}
diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java
index 51beda969465..88d58673706c 100644
--- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java
+++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/KafkaLookupExtractorFactoryTest.java
@@ -299,7 +299,7 @@ public void testStartStop()
EasyMock.expectLastCall().andAnswer(new IAnswer()
{
@Override
- public Object answer() throws Throwable
+ public Object answer()
{
threadWasInterrupted.set(Thread.currentThread().isInterrupted());
return null;
@@ -332,7 +332,7 @@ ConsumerConnector buildConnector(Properties properties)
@Test
- public void testStartFailsFromTimeout() throws Exception
+ public void testStartFailsFromTimeout()
{
EasyMock.expect(cacheManager.createCache())
.andReturn(cacheHandler)
diff --git a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java
index a633f28b79e0..bd08ebe58863 100644
--- a/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java
+++ b/extensions-core/kafka-extraction-namespace/src/test/java/io/druid/query/lookup/TestKafkaExtractionCluster.java
@@ -108,7 +108,7 @@ public void close() throws IOException
closer.register(new Closeable()
{
@Override
- public void close() throws IOException
+ public void close()
{
zkClient.close();
}
@@ -168,7 +168,7 @@ public void sleep(long ms)
closer.register(new Closeable()
{
@Override
- public void close() throws IOException
+ public void close()
{
kafkaServer.shutdown();
kafkaServer.awaitShutdown();
@@ -194,7 +194,7 @@ public void close() throws IOException
try (final AutoCloseable autoCloseable = new AutoCloseable()
{
@Override
- public void close() throws Exception
+ public void close()
{
if (zkClient.exists(zkKafkaPath)) {
try {
@@ -223,7 +223,7 @@ public void close() throws Exception
try (final AutoCloseable autoCloseable = new AutoCloseable()
{
@Override
- public void close() throws Exception
+ public void close()
{
producer.close();
}
@@ -281,7 +281,7 @@ public void configure(Binder binder)
closer.register(new Closeable()
{
@Override
- public void close() throws IOException
+ public void close()
{
factory.close();
}
@@ -322,7 +322,7 @@ public void testSimpleRename() throws InterruptedException
closer.register(new Closeable()
{
@Override
- public void close() throws IOException
+ public void close()
{
producer.close();
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
index 6ebf4a835001..a325948a6506 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java
@@ -84,9 +84,9 @@
import io.druid.segment.realtime.appenderator.Appenderator;
import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
import io.druid.segment.realtime.appenderator.Appenderators;
-import io.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
import io.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import io.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import io.druid.segment.realtime.firehose.ChatHandler;
import io.druid.segment.realtime.firehose.ChatHandlerProvider;
@@ -319,7 +319,7 @@ public String getType()
}
@Override
- public boolean isReady(TaskActionClient taskActionClient) throws Exception
+ public boolean isReady(TaskActionClient taskActionClient)
{
return true;
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
index 6525d1276318..c869ed4665bd 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java
@@ -29,22 +29,22 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
-import io.druid.java.util.emitter.EmittingLogger;
-import io.druid.java.util.http.client.HttpClient;
-import io.druid.java.util.http.client.Request;
-import io.druid.java.util.http.client.response.FullResponseHandler;
-import io.druid.java.util.http.client.response.FullResponseHolder;
+import io.druid.indexer.TaskLocation;
import io.druid.indexing.common.RetryPolicy;
import io.druid.indexing.common.RetryPolicyConfig;
import io.druid.indexing.common.RetryPolicyFactory;
import io.druid.indexing.common.TaskInfoProvider;
-import io.druid.indexer.TaskLocation;
import io.druid.indexing.common.TaskStatus;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.concurrent.Execs;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.FullResponseHandler;
+import io.druid.java.util.http.client.response.FullResponseHolder;
import io.druid.segment.realtime.firehose.ChatHandlerResource;
import org.jboss.netty.channel.ChannelException;
import org.jboss.netty.handler.codec.http.HttpMethod;
@@ -351,7 +351,7 @@ public ListenableFuture stopAsync(final String id, final boolean publis
new Callable()
{
@Override
- public Boolean call() throws Exception
+ public Boolean call()
{
return stop(id, publish);
}
@@ -365,7 +365,7 @@ public ListenableFuture resumeAsync(final String id)
new Callable()
{
@Override
- public Boolean call() throws Exception
+ public Boolean call()
{
return resume(id);
}
@@ -384,7 +384,7 @@ public ListenableFuture> pauseAsync(final String id, final lo
new Callable>()
{
@Override
- public Map call() throws Exception
+ public Map call()
{
return pause(id, timeout);
}
@@ -398,7 +398,7 @@ public ListenableFuture getStatusAsync(final String id)
new Callable()
{
@Override
- public KafkaIndexTask.Status call() throws Exception
+ public KafkaIndexTask.Status call()
{
return getStatus(id);
}
@@ -412,7 +412,7 @@ public ListenableFuture getStartTimeAsync(final String id)
new Callable()
{
@Override
- public DateTime call() throws Exception
+ public DateTime call()
{
return getStartTime(id);
}
@@ -426,7 +426,7 @@ public ListenableFuture> getCurrentOffsetsAsync(final String
new Callable>()
{
@Override
- public Map call() throws Exception
+ public Map call()
{
return getCurrentOffsets(id, retry);
}
@@ -440,7 +440,7 @@ public ListenableFuture> getEndOffsetsAsync(final String id)
new Callable>()
{
@Override
- public Map call() throws Exception
+ public Map call()
{
return getEndOffsets(id);
}
@@ -456,7 +456,7 @@ public ListenableFuture setEndOffsetsAsync(
new Callable()
{
@Override
- public Boolean call() throws Exception
+ public Boolean call()
{
return setEndOffsets(id, endOffsets, resume, finalize);
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
index eee4dc45fcf0..13388e8de089 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java
@@ -632,7 +632,7 @@ private class CheckpointNotice implements Notice
}
@Override
- public void handle() throws ExecutionException, InterruptedException, TimeoutException
+ public void handle() throws ExecutionException, InterruptedException
{
// check for consistency
// if already received request for this sequenceName and dataSourceMetadata combination then return
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
index 2834cc838a89..401c98d8f186 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java
@@ -27,17 +27,17 @@
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
-import io.druid.java.util.http.client.HttpClient;
-import io.druid.java.util.http.client.Request;
-import io.druid.java.util.http.client.response.FullResponseHandler;
-import io.druid.java.util.http.client.response.FullResponseHolder;
-import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexer.TaskLocation;
+import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexing.common.TaskStatus;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.http.client.HttpClient;
+import io.druid.java.util.http.client.Request;
+import io.druid.java.util.http.client.response.FullResponseHandler;
+import io.druid.java.util.http.client.response.FullResponseHolder;
import org.easymock.Capture;
import org.easymock.CaptureType;
import org.easymock.EasyMockSupport;
@@ -54,7 +54,6 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import java.io.IOException;
import java.net.URL;
import java.util.List;
import java.util.Map;
@@ -99,7 +98,7 @@ public KafkaIndexTaskClientTest(int numThreads)
}
@Before
- public void setUp() throws Exception
+ public void setUp()
{
httpClient = createMock(HttpClient.class);
taskInfoProvider = createMock(TaskInfoProvider.class);
@@ -123,13 +122,13 @@ public void setUp() throws Exception
}
@After
- public void tearDown() throws Exception
+ public void tearDown()
{
client.close();
}
@Test
- public void testNoTaskLocation() throws Exception
+ public void testNoTaskLocation()
{
reset(taskInfoProvider);
expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
@@ -151,7 +150,7 @@ public void testNoTaskLocation() throws Exception
}
@Test(expected = KafkaIndexTaskClient.TaskNotRunnableException.class)
- public void testTaskNotRunnableException() throws Exception
+ public void testTaskNotRunnableException()
{
reset(taskInfoProvider);
expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT))
@@ -164,7 +163,7 @@ public void testTaskNotRunnableException() throws Exception
}
@Test(expected = RuntimeException.class)
- public void testInternalServerError() throws Exception
+ public void testInternalServerError()
{
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2);
expect(
@@ -183,7 +182,7 @@ public void testInternalServerError() throws Exception
}
@Test(expected = IAE.class)
- public void testBadRequest() throws Exception
+ public void testBadRequest()
{
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2);
expect(responseHolder.getContent()).andReturn("");
@@ -203,7 +202,7 @@ public void testBadRequest() throws Exception
}
@Test
- public void testTaskLocationMismatch() throws Exception
+ public void testTaskLocationMismatch()
{
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3)
.andReturn(HttpResponseStatus.OK);
@@ -295,7 +294,7 @@ public void testGetCurrentOffsetsWithRetry() throws Exception
}
@Test(expected = RuntimeException.class)
- public void testGetCurrentOffsetsWithExhaustedRetries() throws Exception
+ public void testGetCurrentOffsetsWithExhaustedRetries()
{
client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2);
@@ -993,7 +992,7 @@ public TestableKafkaIndexTaskClient(
}
@Override
- void checkConnection(String host, int port) throws IOException
+ void checkConnection(String host, int port)
{
}
}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
index dfc20525513b..13452ce42276 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -23,7 +23,6 @@
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
-import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables;
@@ -38,10 +37,6 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
-import io.druid.java.util.emitter.EmittingLogger;
-import io.druid.java.util.emitter.core.NoopEmitter;
-import io.druid.java.util.emitter.service.ServiceEmitter;
-import io.druid.java.util.metrics.MonitorScheduler;
import io.druid.client.cache.CacheConfig;
import io.druid.client.cache.MapCache;
import io.druid.data.input.impl.DimensionsSpec;
@@ -84,6 +79,10 @@
import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.common.parsers.JSONPathFieldSpec;
import io.druid.java.util.common.parsers.JSONPathSpec;
+import io.druid.java.util.emitter.EmittingLogger;
+import io.druid.java.util.emitter.core.NoopEmitter;
+import io.druid.java.util.emitter.service.ServiceEmitter;
+import io.druid.java.util.metrics.MonitorScheduler;
import io.druid.math.expr.ExprMacroTable;
import io.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
import io.druid.metadata.EntryExistsException;
@@ -1720,7 +1719,7 @@ private ListenableFuture runTask(final Task task)
new Callable()
{
@Override
- public TaskStatus call() throws Exception
+ public TaskStatus call()
{
try {
if (task.isReady(toolbox.getTaskActionClient())) {
@@ -2045,23 +2044,14 @@ private void destroyToolboxFactory()
metadataStorageCoordinator = null;
}
- private Set publishedDescriptors() throws IOException
+ private Set publishedDescriptors()
{
return FluentIterable.from(
metadataStorageCoordinator.getUsedSegmentsForInterval(
DATA_SCHEMA.getDataSource(),
Intervals.of("0000/3000")
)
- ).transform(
- new Function()
- {
- @Override
- public SegmentDescriptor apply(DataSegment input)
- {
- return input.toDescriptor();
- }
- }
- ).toSet();
+ ).transform(DataSegment::toDescriptor).toSet();
}
private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task)
@@ -2121,7 +2111,7 @@ private List readSegmentColumn(final String column, final SegmentDescrip
return values;
}
- public long countEvents(final Task task) throws Exception
+ public long countEvents(final Task task)
{
// Do a query.
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java
index 5378e4fc2464..d18a34651de3 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java
@@ -99,7 +99,7 @@ public void testSerdeWithNonDefaults() throws Exception
}
@Test
- public void testCopyOf() throws Exception
+ public void testCopyOf()
{
KafkaTuningConfig original = new KafkaTuningConfig(
1,
diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
index 6355d9d83280..a152fff6c685 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
@@ -174,7 +174,7 @@ public static void setupClass() throws Exception
}
@Before
- public void setupTest() throws Exception
+ public void setupTest()
{
taskStorage = createMock(TaskStorage.class);
taskMaster = createMock(TaskMaster.class);
@@ -208,7 +208,7 @@ public void setupTest() throws Exception
}
@After
- public void tearDownTest() throws Exception
+ public void tearDownTest()
{
supervisor = null;
}
@@ -1586,14 +1586,14 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception
}
@Test(expected = IllegalStateException.class)
- public void testStopNotStarted() throws Exception
+ public void testStopNotStarted()
{
supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false);
supervisor.stop(false);
}
@Test
- public void testStop() throws Exception
+ public void testStop()
{
expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
taskClient.close();
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java
index 0c472bf30b37..c0b768d3cb7c 100644
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java
+++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/data/input/MapPopulator.java
@@ -86,7 +86,7 @@ public PopulateResult populate(final ByteSource source, final Map map) thr
private int entries = 0;
@Override
- public boolean processLine(String line) throws IOException
+ public boolean processLine(String line)
{
if (lines == Integer.MAX_VALUE) {
throw new ISE("Cannot read more than %,d lines", Integer.MAX_VALUE);
diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java
index 8084f9eb183e..81c3c9231a7d 100644
--- a/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java
+++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/server/lookup/namespace/JdbcCacheGenerator.java
@@ -19,6 +19,7 @@
package io.druid.server.lookup.namespace;
+import com.google.common.base.Strings;
import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.Pair;
import io.druid.java.util.common.StringUtils;
@@ -33,8 +34,6 @@
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import org.skife.jdbi.v2.util.TimestampMapper;
-import com.google.common.base.Strings;
-
import javax.annotation.Nullable;
import java.sql.ResultSet;
import java.sql.SQLException;
@@ -79,7 +78,7 @@ public CacheScheduler.VersionedCache generateCache(
new HandleCallback>>()
{
@Override
- public List