Prohibit Throwables.propagate() (#7121)

* Throw caught exception.

* Throw caught exceptions.

* Related checkstyle rule is added to prevent further bugs.

* RuntimeException() is used instead of Throwables.propagate().

* Missing import is added.

* Throwables are propogated if possible.

* Throwables are propogated if possible.

* Throwables are propogated if possible.

* Throwables are propogated if possible.

* * Checkstyle definition is improved.
* Throwables.propagate() usages are removed.

* Checkstyle pattern is changed for only scanning "Throwables.propagate(" instead of checking lookbehind.

* Throwable is kept before firing a Runtime Exception.

* Fix unused assignments.
This commit is contained in:
Furkan KAMACI 2019-03-15 00:28:33 +03:00 committed by Roman Leventov
parent e11364883c
commit 7ada1c49f9
229 changed files with 462 additions and 646 deletions

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import com.google.common.io.Files;
import org.apache.commons.io.FileUtils;
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
@ -442,7 +441,7 @@ public class GroupByTypeInterfaceBenchmark
}
catch (IOException e) {
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.benchmark.datagen;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.io.Files;
@ -170,7 +169,7 @@ public class SegmentGenerator implements Closeable
return merged;
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files;
@ -535,7 +534,7 @@ public class GroupByBenchmark
}
catch (IOException e) {
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -204,6 +204,12 @@
<property name="message" value="Use org.apache.druid.server.http.HttpMediaType#TEXT_PLAIN_UTF8 instead"/>
</module>
<module name="Regexp">
<property name="format" value='^Throwables.propagate\('/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Throwables.propagate() shouldn't be used in new code"/>
</module>
<module name="PackageName">
<property name="format" value="^org.apache.druid.*$"/>
</module>

View File

@ -20,7 +20,6 @@
package org.apache.druid.collections;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Ordering;
import org.apache.druid.java.util.common.guava.Accumulator;
import org.apache.druid.java.util.common.guava.CloseQuietly;
@ -116,7 +115,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
retVal.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
return null;
} else {
@ -165,7 +164,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
yielder.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
} else {
pQueue.add(yielder);

View File

@ -19,7 +19,6 @@
package org.apache.druid.collections;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import sun.misc.Cleaner;
@ -137,7 +136,7 @@ public class ReferenceCountingResourceHolder<T> implements ResourceHolder<T>
closer.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.common.config;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
@ -159,10 +158,10 @@ public class ConfigManager
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (ExecutionException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.common.config;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import org.apache.druid.audit.AuditEntry;
import org.apache.druid.audit.AuditInfo;
@ -94,7 +93,7 @@ public class JacksonConfigManager
return jsonMapper.writeValueAsBytes(obj);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -105,7 +104,7 @@ public class JacksonConfigManager
return jsonMapper.writeValueAsString(obj);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -120,7 +119,7 @@ public class JacksonConfigManager
return jsonMapper.readValue(bytes, clazz);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
};
@ -137,7 +136,7 @@ public class JacksonConfigManager
return jsonMapper.writeValueAsBytes(obj);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -148,7 +147,7 @@ public class JacksonConfigManager
return jsonMapper.writeValueAsString(obj);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -163,7 +162,7 @@ public class JacksonConfigManager
return jsonMapper.readValue(bytes, clazz);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
};

View File

@ -19,7 +19,6 @@
package org.apache.druid.common.config;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.ISE;
import org.apache.logging.log4j.core.LifeCycle;
import org.apache.logging.log4j.core.util.Cancellable;
@ -184,7 +183,7 @@ public class Log4jShutdown implements ShutdownCallbackRegistry, LifeCycle
}
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
return current;

View File

@ -20,7 +20,6 @@
package org.apache.druid.data.input.impl;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import org.apache.commons.io.IOUtils;
import org.apache.commons.io.LineIterator;
@ -82,7 +81,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
"Exception reading object[%s]",
object
);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
},

View File

@ -20,7 +20,6 @@
package org.apache.druid.data.input.impl.prefetch;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
@ -224,7 +223,7 @@ public abstract class Fetcher<T> implements Iterator<OpenedObject<T>>
}
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
final FetchedFile<T> maybeCached = cacheIfPossible(fetchedFile);
@ -252,7 +251,7 @@ public abstract class Fetcher<T> implements Iterator<OpenedObject<T>>
return new OpenedObject<>(cached);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
} else {
final T object = objects.get(nextFetchIndex);

View File

@ -27,7 +27,6 @@ import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
import com.google.inject.ProvisionException;
@ -167,7 +166,7 @@ public class JsonConfigurator
}
}
catch (NoSuchFieldException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
messages.add(StringUtils.format("%s - %s", path.toString(), violation.getMessage()));

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.common;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.io.ByteSource;
import com.google.common.io.Files;
@ -85,7 +84,7 @@ public class FileUtils
return new FileCopyResult(outFile);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -93,7 +93,7 @@ public class RetryUtils
awaitNextRetry(e, messageOnRetry, nTry, maxRetries, nTry <= quietTries);
} else {
Throwables.propagateIfInstanceOf(e, Exception.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.common;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSink;
import com.google.common.io.ByteSource;
import com.google.common.io.ByteStreams;
@ -118,7 +117,7 @@ public class StreamUtils
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.common;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import javax.annotation.Nullable;
import java.io.UnsupportedEncodingException;
@ -80,7 +79,7 @@ public class StringUtils
}
catch (UnsupportedEncodingException e) {
// Should never happen
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -103,7 +102,7 @@ public class StringUtils
}
catch (UnsupportedEncodingException e) {
// Should never happen
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.java.util.common.concurrent;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
import java.util.concurrent.ExecutorService;
@ -47,7 +46,7 @@ public class ExecutorServices
);
}
catch (Exception e) {
Throwables.propagate(e);
throw new RuntimeException(e);
}
return service;
}

View File

@ -19,8 +19,6 @@
package org.apache.druid.java.util.common.guava;
import com.google.common.base.Throwables;
import java.io.Closeable;
import java.io.IOException;
@ -93,7 +91,7 @@ public class ConcatSequence<T> implements Sequence<T>
yielder.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
yielderYielder = yielderYielder.next(null);
@ -114,7 +112,7 @@ public class ConcatSequence<T> implements Sequence<T>
yielder.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
return makeYielder(yielderYielder.next(null), nextInit, accumulator);

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.common.guava;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Ordering;
import org.apache.druid.java.util.common.io.Closer;
@ -110,7 +109,7 @@ public class MergeSequence<T> extends YieldingSequenceBase<T>
yielder.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
} else {
pQueue.add(yielder);

View File

@ -59,14 +59,15 @@ final class WrappingSequence<T> implements Sequence<T>
catch (Exception e) {
t.addSuppressed(e);
}
throw Throwables.propagate(t);
Throwables.propagateIfPossible(t);
throw new RuntimeException(t);
}
// "Normal" close
try {
wrapper.after(true, null);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
return result;
}
@ -96,7 +97,8 @@ final class WrappingSequence<T> implements Sequence<T>
catch (Exception e) {
t.addSuppressed(e);
}
throw Throwables.propagate(t);
Throwables.propagateIfPossible(t);
throw new RuntimeException(t);
}
}
}

View File

@ -62,7 +62,7 @@ final class WrappingYielder<OutType> implements Yielder<OutType>
catch (Exception e) {
t.addSuppressed(e);
}
throw Throwables.propagate(t);
throw new RuntimeException(t);
}
}
@ -89,7 +89,7 @@ final class WrappingYielder<OutType> implements Yielder<OutType>
t.addSuppressed(e);
}
Throwables.propagateIfInstanceOf(t, IOException.class);
throw Throwables.propagate(t);
throw new RuntimeException(t);
}
// "Normal" close
try {
@ -97,7 +97,7 @@ final class WrappingYielder<OutType> implements Yielder<OutType>
}
catch (Exception e) {
Throwables.propagateIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -74,7 +74,7 @@ public class Yielders
}
catch (Exception e) {
Throwables.propagateIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.primitives.Ints;
import io.netty.handler.codec.http.HttpHeaders;
import org.apache.druid.concurrent.ConcurrentAwaitableCounter;
@ -276,7 +275,7 @@ public class HttpPostEmitter implements Flushable, Closeable, Emitter
return jsonMapper.writeValueAsBytes(event);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.emitter.core;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
@ -177,7 +176,7 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
if (thrown != null) {
e.addSuppressed(thrown);
}
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
if (thrown == null) {
thrown = e;
@ -189,7 +188,7 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
}
}
if (thrown != null) {
throw Throwables.propagate(thrown);
throw new RuntimeException(thrown);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.java.util.http.client;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.guava.CloseQuietly;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
@ -106,7 +105,7 @@ public class HttpClientInit
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -151,22 +150,22 @@ public class HttpClientInit
return sslContext;
}
catch (CertificateException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (NoSuchAlgorithmException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (KeyStoreException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (KeyManagementException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (FileNotFoundException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
CloseQuietly.close(in);
@ -223,7 +222,7 @@ public class HttpClientInit
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
return bootstrap;

View File

@ -79,7 +79,7 @@ public class ResourcePool<K, V> implements Closeable
holder = pool.get(key);
}
catch (ExecutionException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
final V value = holder.get();
@ -227,7 +227,8 @@ public class ResourcePool<K, V> implements Closeable
deficit++;
this.notifyAll();
}
throw Throwables.propagate(e);
Throwables.propagateIfPossible(e);
throw new RuntimeException(e);
}
return retVal;

View File

@ -19,7 +19,6 @@
package org.apache.druid.java.util.http.client.response;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import org.apache.druid.java.util.common.logger.Logger;
import org.jboss.netty.buffer.ChannelBuffer;
@ -63,7 +62,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
catch (InterruptedException e) {
log.error(e, "Queue appending interrupted");
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
byteCount.addAndGet(response.getContent().readableBytes());
return ClientResponse.finished(
@ -89,7 +88,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
catch (InterruptedException e) {
log.warn(e, "Thread interrupted while taking from queue");
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -115,7 +114,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
catch (InterruptedException e) {
log.warn(e, "Thread interrupted while adding to queue");
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
byteCount.addAndGet(bytes);
} else {
@ -137,12 +136,12 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
catch (InterruptedException e) {
log.warn(e, "Thread interrupted while adding to queue");
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (IOException e) {
// This should never happen
log.wtf(e, "The empty stream threw an IOException");
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
log.debug("Done after adding %d bytes of streams", byteCount.get());

View File

@ -19,7 +19,6 @@
package org.apache.druid.java.util.metrics;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.StreamUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.hyperic.jni.ArchLoaderException;
@ -57,7 +56,7 @@ public class SigarUtil
}
}
catch (ArchNotSupportedException | ArchLoaderException | IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.concurrent;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
@ -89,7 +88,7 @@ public class ExecsTest
taskCompletedSignal.countDown();
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
log.info("Completed task: %s", taskID);
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.java.util.common;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSink;
import com.google.common.io.ByteSource;
import com.google.common.io.ByteStreams;
@ -78,7 +77,7 @@ public class CompressionUtilsTest
}
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
content = builder.toString();
expected = StringUtils.toUtf8(content);
@ -90,7 +89,7 @@ public class CompressionUtilsTest
}
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
gzBytes = gzByteStream.toByteArray();
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.java.util.emitter.core;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.ISE;
import org.asynchttpclient.ListenableFuture;
import org.asynchttpclient.Request;
@ -51,7 +50,7 @@ public abstract class GoHandler
}
catch (Throwable e) {
succeeded = false;
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.storage.azure;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import com.microsoft.azure.storage.StorageException;
@ -55,7 +54,7 @@ public class AzureByteSource extends ByteSource
if (AzureUtils.AZURE_RETRY.apply(e)) {
throw new IOException("Recoverable exception", e);
}
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.storage.azure;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
@ -158,7 +157,7 @@ public class AzureDataSegmentPusher implements DataSegmentPusher
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
if (zipOutFile != null) {

View File

@ -20,7 +20,6 @@
package org.apache.druid.storage.azure;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import com.microsoft.azure.storage.StorageException;
@ -77,7 +76,7 @@ public class AzureTaskLogs implements TaskLogs
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.storage.cloudfiles;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import org.jclouds.io.Payload;
@ -65,7 +64,7 @@ public class CloudFilesByteSource extends ByteSource
if (CloudFilesUtils.CLOUDFILESRETRY.apply(e)) {
throw new IOException("Recoverable exception", e);
}
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.storage.cloudfiles;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.logger.Logger;
@ -124,7 +123,7 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
if (zipOutFile != null) {

View File

@ -22,7 +22,6 @@ package org.apache.druid.storage.google;
import com.google.api.client.http.FileContent;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
@ -127,7 +126,7 @@ public class GoogleDataSegmentPusher implements DataSegmentPusher
return outSegment;
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
if (indexFile != null) {

View File

@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.druid.guice.annotations.Self;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
@ -606,7 +605,7 @@ public class KerberosAuthenticator implements Authenticator
}
}
catch (Exception ex) {
Throwables.propagate(ex);
throw new RuntimeException(ex);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.security.kerberos;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
@ -149,7 +148,7 @@ public class KerberosHttpClient extends AbstractHttpClient
}, exec);
}
catch (Throwable e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.storage.hdfs;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.FileUtils;
@ -215,7 +214,7 @@ public class HdfsDataSegmentPuller implements URIDataPuller
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
} else if (CompressionUtils.isZip(path.getName())) {

View File

@ -19,7 +19,6 @@
package org.apache.druid.storage.hdfs;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import org.apache.druid.data.SearchableVersionedDataFinder;
import org.apache.druid.java.util.common.RetryUtils;
@ -94,7 +93,7 @@ public class HdfsFileTimestampVersionFinder extends HdfsDataSegmentPuller implem
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.storage.hdfs;
import com.fasterxml.jackson.core.Version;
import com.fasterxml.jackson.databind.Module;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Inject;
@ -106,7 +105,7 @@ public class HdfsStorageDruidModule implements DruidModule
FileSystem.get(conf);
}
catch (IOException ex) {
throw Throwables.propagate(ex);
throw new RuntimeException(ex);
}
finally {
Thread.currentThread().setContextClassLoader(currCtxCl);

View File

@ -19,7 +19,6 @@
package /*CHECKSTYLE.OFF: PackageName*/org.apache.hadoop.fs/*CHECKSTYLE.ON: PackageName*/;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.logger.Logger;
import java.io.IOException;
@ -64,7 +63,7 @@ public class HadoopFsWrapper
log.info(ex, "Destination exists while renaming [%s] to [%s]", from, to);
return false;
} else {
throw Throwables.propagate(ex);
throw new RuntimeException(ex);
}
}
catch (NoSuchMethodException | IllegalAccessException ex) {
@ -72,7 +71,7 @@ public class HadoopFsWrapper
for (Method method : fs.getClass().getDeclaredMethods()) {
log.error(method.toGenericString());
}
throw Throwables.propagate(ex);
throw new RuntimeException(ex);
}
}

View File

@ -30,7 +30,6 @@ import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files;
import org.apache.druid.indexer.Bucket;
@ -422,7 +421,7 @@ public class HdfsDataSegmentPusherTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
//DataSchema dataSchema = new DataSchema("dataSource", null, null, Gra)
@ -511,7 +510,7 @@ public class HdfsDataSegmentPusherTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig(

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.primitives.Bytes;
@ -360,7 +359,7 @@ public class KafkaLookupExtractorFactoryTest
Thread.currentThread().join();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
throw new RuntimeException("shouldn't make it here");
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.query.lookup;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Injector;
@ -160,7 +159,7 @@ public class TestKafkaExtractionCluster
Thread.sleep(ms);
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -918,7 +918,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
}
}
catch (Exception e) {
Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -1170,7 +1170,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
return Response.ok().entity(objectMapper.writeValueAsString(getCurrentOffsets())).build();
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables;
import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@ -2882,7 +2881,7 @@ public class KafkaIndexTaskTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexing.kafka;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@ -99,7 +98,7 @@ public class KafkaRecordSupplierTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -3216,7 +3216,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
));
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -29,7 +29,6 @@ import com.amazonaws.services.kinesis.model.Record;
import com.amazonaws.services.kinesis.model.Shard;
import com.amazonaws.services.kinesis.model.StreamDescription;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@ -135,7 +134,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
));
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.kinesis.supervisor;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@ -3582,7 +3581,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
));
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
@ -120,7 +119,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
}
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -132,7 +131,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
writeLock.lockInterruptibly();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
try {
if (entry == null) {
@ -197,7 +196,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
readLock.lockInterruptibly();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
try {
if (entry == null) {

View File

@ -30,7 +30,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.guice.annotations.Json;
@ -619,7 +618,7 @@ public class UriExtractionNamespace implements ExtractionNamespace
return jsonFactory.createParser(input).readValueAs(JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -216,7 +216,7 @@ public final class CacheScheduler
t.addSuppressed(e);
}
if (Thread.currentThread().isInterrupted() || t instanceof InterruptedException || t instanceof Error) {
throw Throwables.propagate(t);
throw new RuntimeException(t);
}
}
}
@ -448,7 +448,7 @@ public final class CacheScheduler
catch (Exception e) {
log.error(e, "Error emitting namespace stats");
if (Thread.currentThread().isInterrupted()) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.server.lookup.namespace.cache;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.concurrent.ExecutorServices;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
@ -73,7 +72,7 @@ public abstract class NamespaceExtractionCacheManager
catch (Exception e) {
log.error(e, "Error emitting namespace stats");
if (Thread.currentThread().isInterrupted()) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -145,7 +145,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
log.info("Using file [%s] for mapDB off heap namespace cache", tmpFile.getAbsolutePath());
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
mmapDB = DBMaker
.newFileDB(tmpFile)
@ -182,7 +182,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.query.lookup.namespace;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
@ -80,7 +79,7 @@ public class JSONFlatDataParserTest
return MAPPER.writeValueAsString(input);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.server.lookup.namespace;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.SearchableVersionedDataFinder;
@ -97,7 +96,7 @@ public class UriCacheGeneratorTest
);
}
catch (URISyntaxException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
return newURI;
}
@ -133,7 +132,7 @@ public class UriCacheGeneratorTest
return new FileOutputStream(outFile);
}
catch (IOException ex) {
throw Throwables.propagate(ex);
throw new RuntimeException(ex);
}
}
}
@ -163,7 +162,7 @@ public class UriCacheGeneratorTest
};
}
catch (IOException ex) {
throw Throwables.propagate(ex);
throw new RuntimeException(ex);
}
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.server.lookup.namespace.cache;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
@ -263,13 +262,13 @@ public class CacheSchedulerTest
testDelete();
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -333,7 +332,7 @@ public class CacheSchedulerTest
}
catch (ExecutionException e) {
if (!future.isCancelled()) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.server.lookup.namespace.cache;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListenableFuture;
@ -189,7 +188,7 @@ public class JdbcExtractionNamespaceTest
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -237,7 +236,7 @@ public class JdbcExtractionNamespaceTest
lifecycle.start();
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
closer.register(
new Closeable()
@ -295,7 +294,7 @@ public class JdbcExtractionNamespaceTest
closer.close();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.server.lookup.namespace.cache;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
@ -92,7 +91,7 @@ public class NamespaceExtractionCacheManagersTest
thunder.await();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
for (int i = 0; i < 1000; ++i) {
CacheHandler cacheHandler = manager.createCache();

View File

@ -142,7 +142,7 @@ public class S3DataSegmentMover implements DataSegmentMover
catch (Exception e) {
Throwables.propagateIfInstanceOf(e, AmazonServiceException.class);
Throwables.propagateIfInstanceOf(e, SegmentLoadingException.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -25,7 +25,6 @@ import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import com.google.common.io.Files;
import com.google.inject.Inject;
@ -100,7 +99,7 @@ public class S3DataSegmentPuller implements URIDataPuller
throw new IOException("Recoverable exception", e);
}
}
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
};
@ -315,7 +314,7 @@ public class S3DataSegmentPuller implements URIDataPuller
throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.storage.s3;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
@ -107,7 +106,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
throw new IOException(e);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
log.info("Deleting temporary cached index.zip");

View File

@ -142,7 +142,7 @@ public class S3TaskLogs implements TaskLogs
}
catch (Exception e) {
Throwables.propagateIfInstanceOf(e, IOException.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.storage.s3;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import org.apache.druid.data.SearchableVersionedDataFinder;
import org.apache.druid.java.util.common.RetryUtils;
@ -93,7 +92,7 @@ public class S3TimestampVersionedDataFinder extends S3DataSegmentPuller implemen
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.indexer;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
@ -220,7 +219,7 @@ public class DetermineHashedPartitionsJob implements Jobby
return true;
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -24,7 +24,6 @@ import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
@ -272,7 +271,7 @@ public class DeterminePartitionsJob implements Jobby
return true;
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -28,7 +28,6 @@ import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.inject.Binder;
@ -168,7 +167,7 @@ public class HadoopDruidIndexerConfig
);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -182,7 +181,7 @@ public class HadoopDruidIndexerConfig
);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -199,7 +198,7 @@ public class HadoopDruidIndexerConfig
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -581,7 +580,7 @@ public class HadoopDruidIndexerConfig
conf.set(HadoopDruidIndexerConfig.CONFIG_PROPERTY, HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(this));
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
@ -129,10 +128,10 @@ public class IndexGeneratorJob implements Jobby
+ " either there was no input data to process or all the input events were discarded due to some error",
e.getMessage()
);
Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
List<DataSegment> publishedSegments = publishedSegmentsBuilder.build();
@ -727,7 +726,7 @@ public class IndexGeneratorJob implements Jobby
}
catch (Exception e) {
log.error(e, "persist index error");
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
// close this index
@ -856,7 +855,7 @@ public class IndexGeneratorJob implements Jobby
}
}
catch (ExecutionException | TimeoutException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
index.close();

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexer;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.io.Files;
import org.apache.druid.indexer.updater.HadoopDruidConverterConfig;
import org.apache.druid.java.util.common.DateTimes;
@ -174,7 +173,7 @@ public class JobHelper
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -344,7 +343,7 @@ public class JobHelper
config.addInputPaths(job);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -673,7 +672,7 @@ public class JobHelper
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -858,7 +857,7 @@ public class JobHelper
}
catch (Exception e) {
log.error(e, "Failed to cleanup path[%s]", path);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexer.hadoop;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.io.Closeables;
@ -104,7 +103,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
);
}
catch (IOException ex) {
throw Throwables.propagate(ex);
throw new RuntimeException(ex);
}
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexer.updater;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
@ -205,7 +204,7 @@ public class HadoopConverterJob
);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -320,13 +319,13 @@ public class HadoopConverterJob
}
}
catch (final IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
try (final InputStream stream = fs.open(input)) {
return HadoopDruidConverterConfig.jsonMapper.readValue(stream, DataSegment.class);
}
catch (final IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -344,9 +343,9 @@ public class HadoopConverterJob
}
}
catch (InterruptedException | ClassNotFoundException e) {
RuntimeException exception = Throwables.propagate(e);
throwable = exception;
throw exception;
RuntimeException r = new RuntimeException(e);
throwable = r;
throw r;
}
catch (Throwable t) {
throwable = t;

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexer;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
@ -103,7 +102,7 @@ public class BatchDeltaIngestionTest
);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexer;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
@ -71,7 +70,7 @@ public class HadoopIngestionSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) schema.getDataSchema().getGranularitySpec();
@ -111,7 +110,7 @@ public class HadoopIngestionSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) schema.getDataSchema().getGranularitySpec();
@ -142,7 +141,7 @@ public class HadoopIngestionSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
final PartitionsSpec partitionsSpec = schema.getTuningConfig().getPartitionsSpec();
@ -188,7 +187,7 @@ public class HadoopIngestionSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
final PartitionsSpec partitionsSpec = schema.getTuningConfig().getPartitionsSpec();
@ -261,7 +260,7 @@ public class HadoopIngestionSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
Assert.assertEquals(
@ -326,7 +325,7 @@ public class HadoopIngestionSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
Assert.assertEquals(
@ -342,7 +341,7 @@ public class HadoopIngestionSpecTest
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.segment.IndexSpec;
@ -89,7 +88,7 @@ public class HadoopTuningConfigTest
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexer.hadoop;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.io.Files;
@ -83,14 +82,14 @@ public class FSSpideringIteratorTest
Assert.assertTrue(files.isEmpty());
}
catch (IOException e) {
Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
try {
FileUtils.deleteDirectory(baseDir);
}
catch (IOException e) {
Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexer.partitions;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.junit.Assert;
@ -48,7 +47,7 @@ public class HashedPartitionsSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
Assert.assertEquals(
@ -94,7 +93,7 @@ public class HashedPartitionsSpecTest
);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
Assert.assertEquals(
@ -136,7 +135,7 @@ public class HashedPartitionsSpecTest
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
@ -91,7 +90,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
return result.isOk() ? result.getTaskLock() : null;
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import org.apache.druid.discovery.DruidLeaderClient;
import org.apache.druid.indexing.common.RetryPolicy;
import org.apache.druid.indexing.common.RetryPolicyFactory;
@ -109,7 +108,7 @@ public class RemoteTaskActionClient implements TaskActionClient
Thread.sleep(sleepTime);
}
catch (InterruptedException e2) {
throw Throwables.propagate(e2);
throw new RuntimeException(e2);
}
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.indexing.common.TaskLockType;
@ -197,7 +196,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
} else {
log.error(

View File

@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.commons.io.FileUtils;
@ -209,7 +208,7 @@ public class YeOldePlumberSchool implements PlumberSchool
}
catch (Exception e) {
log.warn(e, "Failed to merge and upload");
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
try {
@ -250,7 +249,7 @@ public class YeOldePlumberSchool implements PlumberSchool
}
catch (Exception e) {
log.warn(e, "Failed to spill index[%d]", indexToPersist.getCount());
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -425,7 +425,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
}
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.indexing.common.task;
import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.inject.Injector;
@ -92,7 +91,7 @@ public abstract class HadoopTask extends AbstractTask
return fName.startsWith("druid") && fName.endsWith(".jar") && !fName.endsWith("selfcontained.jar");
}
catch (URISyntaxException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
};
@ -217,7 +216,7 @@ public abstract class HadoopTask extends AbstractTask
return (OutputType) method.invoke(null, input);
}
catch (IllegalAccessException | InvocationTargetException | ClassNotFoundException | NoSuchMethodException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
Thread.currentThread().setContextClassLoader(oldLoader);
@ -245,7 +244,7 @@ public abstract class HadoopTask extends AbstractTask
return clazz.newInstance();
}
catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
Thread.currentThread().setContextClassLoader(oldLoader);

View File

@ -1069,7 +1069,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
}
}
catch (TimeoutException | ExecutionException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.druid.indexing.common.TaskToolbox;
@ -86,7 +85,7 @@ public class MergeTask extends MergeTaskBase
return toolbox.getIndexIO().loadIndex(input);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.io.FileUtils;
@ -313,7 +312,7 @@ public class RealtimeIndexTask extends AbstractTask
return lock.getVersion();
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
};
@ -517,7 +516,7 @@ public class RealtimeIndexTask extends AbstractTask
}
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.BiMap;
import com.google.common.collect.HashBiMap;
import com.google.common.collect.Iterables;
@ -225,7 +224,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
);
}
catch (IOException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -240,7 +239,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
}
catch (SegmentLoadingException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -27,7 +27,6 @@ import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
@ -474,7 +473,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
catch (Throwable t) {
log.info(t, "Exception caught during execution");
throw Throwables.propagate(t);
throw new RuntimeException(t);
}
finally {
try {

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
@ -146,7 +145,7 @@ public class MetadataTaskStorage implements TaskStorage
if (e instanceof EntryExistsException) {
throw (EntryExistsException) e;
} else {
Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -340,7 +340,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
lifecycleLock.started();
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
lifecycleLock.exitStart();
@ -378,7 +378,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
lifecycleLock.exitStop();
@ -617,12 +617,12 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
).get();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (ExecutionException e) {
// Unwrap if possible
Throwables.propagateIfPossible(e.getCause(), IOException.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -738,7 +738,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
log.info("Tried to delete status path[%s] that didn't exist! Must've gone away already?", statusPath);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
@ -1047,7 +1047,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
return retVal;
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
@ -1086,7 +1086,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
scheduleTasksCleanupForWorker(worker.getHost(), getAssignedTasks(worker));
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
try {
@ -1149,7 +1149,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
catch (Exception e) {
log.makeAlert("Exception while cleaning up worker[%s]", worker).emit();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
},
@ -1263,7 +1263,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
return getWorkerFromZK(lazyWorkers.values());

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexing.overlord;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import org.apache.druid.client.indexing.IndexingService;
import org.apache.druid.curator.discovery.ServiceAnnouncer;
@ -152,7 +151,7 @@ public class TaskMaster implements TaskCountStatsProvider
leaderLifecycle.start();
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
giant.unlock();

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.overlord;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
@ -585,7 +584,7 @@ public class TaskQueue
}
catch (Exception e) {
log.warn(e, "Failed to sync tasks from storage!");
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
giant.unlock();

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexing.overlord;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.worker.Worker;
@ -111,7 +110,7 @@ public class TaskRunnerUtils
return new URI(StringUtils.format("%s://%s%s", worker.getScheme(), worker.getHost(), path)).toURL();
}
catch (URISyntaxException | MalformedURLException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
@ -70,7 +69,7 @@ public class ZkWorker implements Closeable
return jsonMapper.readValue(input.getData(), TaskAnnouncement.class);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
};

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.common.StringUtils;
@ -81,7 +80,7 @@ public class GalaxyEC2UserData implements EC2UserData<GalaxyEC2UserData>
return StringUtils.encodeBase64String(jsonMapper.writeValueAsBytes(this));
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

View File

@ -248,7 +248,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
log.info("Started.");
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
lifecycleLock.exitStart();
@ -539,7 +539,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
scheduleTasksCleanupForWorker(worker.getHost());
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
finally {
checkAndRemoveWorkersFromBlackList();
@ -597,7 +597,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
catch (Exception e) {
log.makeAlert("Exception while cleaning up worker[%s]", workerHostAndPort).emit();
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
},
@ -792,7 +792,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
return getLazyWorkers();
@ -871,12 +871,12 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
).get();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
catch (ExecutionException e) {
// Unwrap if possible
Throwables.propagateIfPossible(e.getCause(), IOException.class);
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.indexing.overlord.http;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Inject;
import org.apache.druid.indexing.overlord.TaskMaster;
@ -70,7 +69,7 @@ public class OverlordRedirectInfo implements RedirectInfo
return new URL(location);
}
catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
@ -68,7 +67,7 @@ public class JavaScriptWorkerSelectStrategy implements WorkerSelectStrategy
return ((Invocable) engine).getInterface(SelectorFunction.class);
}
catch (ScriptException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

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