mirror of https://github.com/apache/druid.git
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:
parent
e11364883c
commit
7ada1c49f9
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Suppliers;
|
import com.google.common.base.Suppliers;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||||
|
@ -442,7 +441,7 @@ public class GroupByTypeInterfaceBenchmark
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
|
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.benchmark.datagen;
|
package org.apache.druid.benchmark.datagen;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
@ -170,7 +169,7 @@ public class SegmentGenerator implements Closeable
|
||||||
return merged;
|
return merged;
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Suppliers;
|
import com.google.common.base.Suppliers;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
@ -535,7 +534,7 @@ public class GroupByBenchmark
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
|
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -204,6 +204,12 @@
|
||||||
<property name="message" value="Use org.apache.druid.server.http.HttpMediaType#TEXT_PLAIN_UTF8 instead"/>
|
<property name="message" value="Use org.apache.druid.server.http.HttpMediaType#TEXT_PLAIN_UTF8 instead"/>
|
||||||
</module>
|
</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">
|
<module name="PackageName">
|
||||||
<property name="format" value="^org.apache.druid.*$"/>
|
<property name="format" value="^org.apache.druid.*$"/>
|
||||||
</module>
|
</module>
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.collections;
|
package org.apache.druid.collections;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import org.apache.druid.java.util.common.guava.Accumulator;
|
import org.apache.druid.java.util.common.guava.Accumulator;
|
||||||
import org.apache.druid.java.util.common.guava.CloseQuietly;
|
import org.apache.druid.java.util.common.guava.CloseQuietly;
|
||||||
|
@ -116,7 +115,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
|
||||||
retVal.close();
|
retVal.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
} else {
|
} else {
|
||||||
|
@ -165,7 +164,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
|
||||||
yielder.close();
|
yielder.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
pQueue.add(yielder);
|
pQueue.add(yielder);
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.collections;
|
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.ISE;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import sun.misc.Cleaner;
|
import sun.misc.Cleaner;
|
||||||
|
@ -137,7 +136,7 @@ public class ReferenceCountingResourceHolder<T> implements ResourceHolder<T>
|
||||||
closer.close();
|
closer.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.common.config;
|
package org.apache.druid.common.config;
|
||||||
|
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
|
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
|
||||||
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
|
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
|
||||||
|
@ -159,10 +158,10 @@ public class ConfigManager
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (ExecutionException e) {
|
catch (ExecutionException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.common.config;
|
||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.audit.AuditEntry;
|
import org.apache.druid.audit.AuditEntry;
|
||||||
import org.apache.druid.audit.AuditInfo;
|
import org.apache.druid.audit.AuditInfo;
|
||||||
|
@ -94,7 +93,7 @@ public class JacksonConfigManager
|
||||||
return jsonMapper.writeValueAsBytes(obj);
|
return jsonMapper.writeValueAsBytes(obj);
|
||||||
}
|
}
|
||||||
catch (JsonProcessingException e) {
|
catch (JsonProcessingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -105,7 +104,7 @@ public class JacksonConfigManager
|
||||||
return jsonMapper.writeValueAsString(obj);
|
return jsonMapper.writeValueAsString(obj);
|
||||||
}
|
}
|
||||||
catch (JsonProcessingException e) {
|
catch (JsonProcessingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -120,7 +119,7 @@ public class JacksonConfigManager
|
||||||
return jsonMapper.readValue(bytes, clazz);
|
return jsonMapper.readValue(bytes, clazz);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -137,7 +136,7 @@ public class JacksonConfigManager
|
||||||
return jsonMapper.writeValueAsBytes(obj);
|
return jsonMapper.writeValueAsBytes(obj);
|
||||||
}
|
}
|
||||||
catch (JsonProcessingException e) {
|
catch (JsonProcessingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -148,7 +147,7 @@ public class JacksonConfigManager
|
||||||
return jsonMapper.writeValueAsString(obj);
|
return jsonMapper.writeValueAsString(obj);
|
||||||
}
|
}
|
||||||
catch (JsonProcessingException e) {
|
catch (JsonProcessingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -163,7 +162,7 @@ public class JacksonConfigManager
|
||||||
return jsonMapper.readValue(bytes, clazz);
|
return jsonMapper.readValue(bytes, clazz);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.common.config;
|
package org.apache.druid.common.config;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.logging.log4j.core.LifeCycle;
|
import org.apache.logging.log4j.core.LifeCycle;
|
||||||
import org.apache.logging.log4j.core.util.Cancellable;
|
import org.apache.logging.log4j.core.util.Cancellable;
|
||||||
|
@ -184,7 +183,7 @@ public class Log4jShutdown implements ShutdownCallbackRegistry, LifeCycle
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return current;
|
return current;
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.data.input.impl;
|
package org.apache.druid.data.input.impl;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.io.LineIterator;
|
import org.apache.commons.io.LineIterator;
|
||||||
|
@ -82,7 +81,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
|
||||||
"Exception reading object[%s]",
|
"Exception reading object[%s]",
|
||||||
object
|
object
|
||||||
);
|
);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.data.input.impl.prefetch;
|
package org.apache.druid.data.input.impl.prefetch;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
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.ISE;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
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) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final FetchedFile<T> maybeCached = cacheIfPossible(fetchedFile);
|
final FetchedFile<T> maybeCached = cacheIfPossible(fetchedFile);
|
||||||
|
@ -252,7 +251,7 @@ public abstract class Fetcher<T> implements Iterator<OpenedObject<T>>
|
||||||
return new OpenedObject<>(cached);
|
return new OpenedObject<>(cached);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
final T object = objects.get(nextFetchIndex);
|
final T object = objects.get(nextFetchIndex);
|
||||||
|
|
|
@ -27,7 +27,6 @@ import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.google.inject.ProvisionException;
|
import com.google.inject.ProvisionException;
|
||||||
|
@ -167,7 +166,7 @@ public class JsonConfigurator
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (NoSuchFieldException e) {
|
catch (NoSuchFieldException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
messages.add(StringUtils.format("%s - %s", path.toString(), violation.getMessage()));
|
messages.add(StringUtils.format("%s - %s", path.toString(), violation.getMessage()));
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.java.util.common;
|
package org.apache.druid.java.util.common;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
@ -85,7 +84,7 @@ public class FileUtils
|
||||||
return new FileCopyResult(outFile);
|
return new FileCopyResult(outFile);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -93,7 +93,7 @@ public class RetryUtils
|
||||||
awaitNextRetry(e, messageOnRetry, nTry, maxRetries, nTry <= quietTries);
|
awaitNextRetry(e, messageOnRetry, nTry, maxRetries, nTry <= quietTries);
|
||||||
} else {
|
} else {
|
||||||
Throwables.propagateIfInstanceOf(e, Exception.class);
|
Throwables.propagateIfInstanceOf(e, Exception.class);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.java.util.common;
|
package org.apache.druid.java.util.common;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSink;
|
import com.google.common.io.ByteSink;
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.common.io.ByteStreams;
|
import com.google.common.io.ByteStreams;
|
||||||
|
@ -118,7 +117,7 @@ public class StreamUtils
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.java.util.common;
|
package org.apache.druid.java.util.common;
|
||||||
|
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
|
@ -80,7 +79,7 @@ public class StringUtils
|
||||||
}
|
}
|
||||||
catch (UnsupportedEncodingException e) {
|
catch (UnsupportedEncodingException e) {
|
||||||
// Should never happen
|
// Should never happen
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -103,7 +102,7 @@ public class StringUtils
|
||||||
}
|
}
|
||||||
catch (UnsupportedEncodingException e) {
|
catch (UnsupportedEncodingException e) {
|
||||||
// Should never happen
|
// Should never happen
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.java.util.common.concurrent;
|
package org.apache.druid.java.util.common.concurrent;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
||||||
|
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
@ -47,7 +46,7 @@ public class ExecutorServices
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
return service;
|
return service;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,8 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.java.util.common.guava;
|
package org.apache.druid.java.util.common.guava;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
@ -93,7 +91,7 @@ public class ConcatSequence<T> implements Sequence<T>
|
||||||
yielder.close();
|
yielder.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
yielderYielder = yielderYielder.next(null);
|
yielderYielder = yielderYielder.next(null);
|
||||||
|
@ -114,7 +112,7 @@ public class ConcatSequence<T> implements Sequence<T>
|
||||||
yielder.close();
|
yielder.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
return makeYielder(yielderYielder.next(null), nextInit, accumulator);
|
return makeYielder(yielderYielder.next(null), nextInit, accumulator);
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.java.util.common.guava;
|
package org.apache.druid.java.util.common.guava;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import org.apache.druid.java.util.common.io.Closer;
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
|
|
||||||
|
@ -110,7 +109,7 @@ public class MergeSequence<T> extends YieldingSequenceBase<T>
|
||||||
yielder.close();
|
yielder.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
pQueue.add(yielder);
|
pQueue.add(yielder);
|
||||||
|
|
|
@ -59,14 +59,15 @@ final class WrappingSequence<T> implements Sequence<T>
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
t.addSuppressed(e);
|
t.addSuppressed(e);
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(t);
|
Throwables.propagateIfPossible(t);
|
||||||
|
throw new RuntimeException(t);
|
||||||
}
|
}
|
||||||
// "Normal" close
|
// "Normal" close
|
||||||
try {
|
try {
|
||||||
wrapper.after(true, null);
|
wrapper.after(true, null);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -96,7 +97,8 @@ final class WrappingSequence<T> implements Sequence<T>
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
t.addSuppressed(e);
|
t.addSuppressed(e);
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(t);
|
Throwables.propagateIfPossible(t);
|
||||||
|
throw new RuntimeException(t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -62,7 +62,7 @@ final class WrappingYielder<OutType> implements Yielder<OutType>
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
t.addSuppressed(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);
|
t.addSuppressed(e);
|
||||||
}
|
}
|
||||||
Throwables.propagateIfInstanceOf(t, IOException.class);
|
Throwables.propagateIfInstanceOf(t, IOException.class);
|
||||||
throw Throwables.propagate(t);
|
throw new RuntimeException(t);
|
||||||
}
|
}
|
||||||
// "Normal" close
|
// "Normal" close
|
||||||
try {
|
try {
|
||||||
|
@ -97,7 +97,7 @@ final class WrappingYielder<OutType> implements Yielder<OutType>
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
Throwables.propagateIfInstanceOf(e, IOException.class);
|
Throwables.propagateIfInstanceOf(e, IOException.class);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -74,7 +74,7 @@ public class Yielders
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
Throwables.propagateIfInstanceOf(e, IOException.class);
|
Throwables.propagateIfInstanceOf(e, IOException.class);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import io.netty.handler.codec.http.HttpHeaders;
|
import io.netty.handler.codec.http.HttpHeaders;
|
||||||
import org.apache.druid.concurrent.ConcurrentAwaitableCounter;
|
import org.apache.druid.concurrent.ConcurrentAwaitableCounter;
|
||||||
|
@ -276,7 +275,7 @@ public class HttpPostEmitter implements Flushable, Closeable, Emitter
|
||||||
return jsonMapper.writeValueAsBytes(event);
|
return jsonMapper.writeValueAsBytes(event);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.java.util.emitter.core;
|
package org.apache.druid.java.util.emitter.core;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
||||||
|
@ -177,7 +176,7 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
|
||||||
if (thrown != null) {
|
if (thrown != null) {
|
||||||
e.addSuppressed(thrown);
|
e.addSuppressed(thrown);
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
if (thrown == null) {
|
if (thrown == null) {
|
||||||
thrown = e;
|
thrown = e;
|
||||||
|
@ -189,7 +188,7 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (thrown != null) {
|
if (thrown != null) {
|
||||||
throw Throwables.propagate(thrown);
|
throw new RuntimeException(thrown);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.java.util.http.client;
|
package org.apache.druid.java.util.http.client;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import org.apache.druid.java.util.common.guava.CloseQuietly;
|
import org.apache.druid.java.util.common.guava.CloseQuietly;
|
||||||
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
||||||
|
@ -106,7 +105,7 @@ public class HttpClientInit
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -151,22 +150,22 @@ public class HttpClientInit
|
||||||
return sslContext;
|
return sslContext;
|
||||||
}
|
}
|
||||||
catch (CertificateException e) {
|
catch (CertificateException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (NoSuchAlgorithmException e) {
|
catch (NoSuchAlgorithmException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (KeyStoreException e) {
|
catch (KeyStoreException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (KeyManagementException e) {
|
catch (KeyManagementException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (FileNotFoundException e) {
|
catch (FileNotFoundException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
CloseQuietly.close(in);
|
CloseQuietly.close(in);
|
||||||
|
@ -223,7 +222,7 @@ public class HttpClientInit
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
return bootstrap;
|
return bootstrap;
|
||||||
|
|
|
@ -79,7 +79,7 @@ public class ResourcePool<K, V> implements Closeable
|
||||||
holder = pool.get(key);
|
holder = pool.get(key);
|
||||||
}
|
}
|
||||||
catch (ExecutionException e) {
|
catch (ExecutionException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
final V value = holder.get();
|
final V value = holder.get();
|
||||||
|
|
||||||
|
@ -227,7 +227,8 @@ public class ResourcePool<K, V> implements Closeable
|
||||||
deficit++;
|
deficit++;
|
||||||
this.notifyAll();
|
this.notifyAll();
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(e);
|
Throwables.propagateIfPossible(e);
|
||||||
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
return retVal;
|
return retVal;
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.java.util.http.client.response;
|
package org.apache.druid.java.util.http.client.response;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.jboss.netty.buffer.ChannelBuffer;
|
import org.jboss.netty.buffer.ChannelBuffer;
|
||||||
|
@ -63,7 +62,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
log.error(e, "Queue appending interrupted");
|
log.error(e, "Queue appending interrupted");
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
byteCount.addAndGet(response.getContent().readableBytes());
|
byteCount.addAndGet(response.getContent().readableBytes());
|
||||||
return ClientResponse.finished(
|
return ClientResponse.finished(
|
||||||
|
@ -89,7 +88,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
log.warn(e, "Thread interrupted while taking from queue");
|
log.warn(e, "Thread interrupted while taking from queue");
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -115,7 +114,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
log.warn(e, "Thread interrupted while adding to queue");
|
log.warn(e, "Thread interrupted while adding to queue");
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
byteCount.addAndGet(bytes);
|
byteCount.addAndGet(bytes);
|
||||||
} else {
|
} else {
|
||||||
|
@ -137,12 +136,12 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
log.warn(e, "Thread interrupted while adding to queue");
|
log.warn(e, "Thread interrupted while adding to queue");
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
// This should never happen
|
// This should never happen
|
||||||
log.wtf(e, "The empty stream threw an IOException");
|
log.wtf(e, "The empty stream threw an IOException");
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
log.debug("Done after adding %d bytes of streams", byteCount.get());
|
log.debug("Done after adding %d bytes of streams", byteCount.get());
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.java.util.metrics;
|
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.StreamUtils;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.hyperic.jni.ArchLoaderException;
|
import org.hyperic.jni.ArchLoaderException;
|
||||||
|
@ -57,7 +56,7 @@ public class SigarUtil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (ArchNotSupportedException | ArchLoaderException | IOException e) {
|
catch (ArchNotSupportedException | ArchLoaderException | IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.concurrent;
|
package org.apache.druid.concurrent;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
@ -89,7 +88,7 @@ public class ExecsTest
|
||||||
taskCompletedSignal.countDown();
|
taskCompletedSignal.countDown();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
log.info("Completed task: %s", taskID);
|
log.info("Completed task: %s", taskID);
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.java.util.common;
|
package org.apache.druid.java.util.common;
|
||||||
|
|
||||||
import com.google.common.base.Predicates;
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSink;
|
import com.google.common.io.ByteSink;
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.common.io.ByteStreams;
|
import com.google.common.io.ByteStreams;
|
||||||
|
@ -78,7 +77,7 @@ public class CompressionUtilsTest
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
content = builder.toString();
|
content = builder.toString();
|
||||||
expected = StringUtils.toUtf8(content);
|
expected = StringUtils.toUtf8(content);
|
||||||
|
@ -90,7 +89,7 @@ public class CompressionUtilsTest
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
gzBytes = gzByteStream.toByteArray();
|
gzBytes = gzByteStream.toByteArray();
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.java.util.emitter.core;
|
package org.apache.druid.java.util.emitter.core;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.asynchttpclient.ListenableFuture;
|
import org.asynchttpclient.ListenableFuture;
|
||||||
import org.asynchttpclient.Request;
|
import org.asynchttpclient.Request;
|
||||||
|
@ -51,7 +50,7 @@ public abstract class GoHandler
|
||||||
}
|
}
|
||||||
catch (Throwable e) {
|
catch (Throwable e) {
|
||||||
succeeded = false;
|
succeeded = false;
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.storage.azure;
|
package org.apache.druid.storage.azure;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.microsoft.azure.storage.StorageException;
|
import com.microsoft.azure.storage.StorageException;
|
||||||
|
|
||||||
|
@ -55,7 +54,7 @@ public class AzureByteSource extends ByteSource
|
||||||
if (AzureUtils.AZURE_RETRY.apply(e)) {
|
if (AzureUtils.AZURE_RETRY.apply(e)) {
|
||||||
throw new IOException("Recoverable exception", e);
|
throw new IOException("Recoverable exception", e);
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.storage.azure;
|
package org.apache.druid.storage.azure;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -158,7 +157,7 @@ public class AzureDataSegmentPusher implements DataSegmentPusher
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
if (zipOutFile != null) {
|
if (zipOutFile != null) {
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.storage.azure;
|
package org.apache.druid.storage.azure;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.microsoft.azure.storage.StorageException;
|
import com.microsoft.azure.storage.StorageException;
|
||||||
|
@ -77,7 +76,7 @@ public class AzureTaskLogs implements TaskLogs
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.storage.cloudfiles;
|
package org.apache.druid.storage.cloudfiles;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import org.jclouds.io.Payload;
|
import org.jclouds.io.Payload;
|
||||||
|
|
||||||
|
@ -65,7 +64,7 @@ public class CloudFilesByteSource extends ByteSource
|
||||||
if (CloudFilesUtils.CLOUDFILESRETRY.apply(e)) {
|
if (CloudFilesUtils.CLOUDFILESRETRY.apply(e)) {
|
||||||
throw new IOException("Recoverable exception", e);
|
throw new IOException("Recoverable exception", e);
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.storage.cloudfiles;
|
package org.apache.druid.storage.cloudfiles;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
@ -124,7 +123,7 @@ public class CloudFilesDataSegmentPusher implements DataSegmentPusher
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
if (zipOutFile != null) {
|
if (zipOutFile != null) {
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.storage.google;
|
||||||
import com.google.api.client.http.FileContent;
|
import com.google.api.client.http.FileContent;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -127,7 +126,7 @@ public class GoogleDataSegmentPusher implements DataSegmentPusher
|
||||||
return outSegment;
|
return outSegment;
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
if (indexFile != null) {
|
if (indexFile != null) {
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.guice.annotations.Self;
|
import org.apache.druid.guice.annotations.Self;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
@ -606,7 +605,7 @@ public class KerberosAuthenticator implements Authenticator
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.security.kerberos;
|
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.FutureCallback;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
@ -149,7 +148,7 @@ public class KerberosHttpClient extends AbstractHttpClient
|
||||||
}, exec);
|
}, exec);
|
||||||
}
|
}
|
||||||
catch (Throwable e) {
|
catch (Throwable e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.storage.hdfs;
|
package org.apache.druid.storage.hdfs;
|
||||||
|
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.java.util.common.FileUtils;
|
import org.apache.druid.java.util.common.FileUtils;
|
||||||
|
@ -215,7 +214,7 @@ public class HdfsDataSegmentPuller implements URIDataPuller
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
} else if (CompressionUtils.isZip(path.getName())) {
|
} else if (CompressionUtils.isZip(path.getName())) {
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.storage.hdfs;
|
package org.apache.druid.storage.hdfs;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.data.SearchableVersionedDataFinder;
|
import org.apache.druid.data.SearchableVersionedDataFinder;
|
||||||
import org.apache.druid.java.util.common.RetryUtils;
|
import org.apache.druid.java.util.common.RetryUtils;
|
||||||
|
@ -94,7 +93,7 @@ public class HdfsFileTimestampVersionFinder extends HdfsDataSegmentPuller implem
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.storage.hdfs;
|
||||||
|
|
||||||
import com.fasterxml.jackson.core.Version;
|
import com.fasterxml.jackson.core.Version;
|
||||||
import com.fasterxml.jackson.databind.Module;
|
import com.fasterxml.jackson.databind.Module;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -106,7 +105,7 @@ public class HdfsStorageDruidModule implements DruidModule
|
||||||
FileSystem.get(conf);
|
FileSystem.get(conf);
|
||||||
}
|
}
|
||||||
catch (IOException ex) {
|
catch (IOException ex) {
|
||||||
throw Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
Thread.currentThread().setContextClassLoader(currCtxCl);
|
Thread.currentThread().setContextClassLoader(currCtxCl);
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package /*CHECKSTYLE.OFF: PackageName*/org.apache.hadoop.fs/*CHECKSTYLE.ON: PackageName*/;
|
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 org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -64,7 +63,7 @@ public class HadoopFsWrapper
|
||||||
log.info(ex, "Destination exists while renaming [%s] to [%s]", from, to);
|
log.info(ex, "Destination exists while renaming [%s] to [%s]", from, to);
|
||||||
return false;
|
return false;
|
||||||
} else {
|
} else {
|
||||||
throw Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (NoSuchMethodException | IllegalAccessException ex) {
|
catch (NoSuchMethodException | IllegalAccessException ex) {
|
||||||
|
@ -72,7 +71,7 @@ public class HadoopFsWrapper
|
||||||
for (Method method : fs.getClass().getDeclaredMethods()) {
|
for (Method method : fs.getClass().getDeclaredMethods()) {
|
||||||
log.error(method.toGenericString());
|
log.error(method.toGenericString());
|
||||||
}
|
}
|
||||||
throw Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||||
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
|
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import org.apache.druid.indexer.Bucket;
|
import org.apache.druid.indexer.Bucket;
|
||||||
|
@ -422,7 +421,7 @@ public class HdfsDataSegmentPusherTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
//DataSchema dataSchema = new DataSchema("dataSource", null, null, Gra)
|
//DataSchema dataSchema = new DataSchema("dataSource", null, null, Gra)
|
||||||
|
@ -511,7 +510,7 @@ public class HdfsDataSegmentPusherTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig(
|
HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig(
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.BeanProperty;
|
||||||
import com.fasterxml.jackson.databind.DeserializationContext;
|
import com.fasterxml.jackson.databind.DeserializationContext;
|
||||||
import com.fasterxml.jackson.databind.InjectableValues;
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.primitives.Bytes;
|
import com.google.common.primitives.Bytes;
|
||||||
|
@ -360,7 +359,7 @@ public class KafkaLookupExtractorFactoryTest
|
||||||
Thread.currentThread().join();
|
Thread.currentThread().join();
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
throw new RuntimeException("shouldn't make it here");
|
throw new RuntimeException("shouldn't make it here");
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.query.lookup;
|
package org.apache.druid.query.lookup;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
|
@ -160,7 +159,7 @@ public class TestKafkaExtractionCluster
|
||||||
Thread.sleep(ms);
|
Thread.sleep(ms);
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -918,7 +918,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner<In
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
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();
|
return Response.ok().entity(objectMapper.writeValueAsString(getCurrentOffsets())).build();
|
||||||
}
|
}
|
||||||
catch (JsonProcessingException e) {
|
catch (JsonProcessingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.Module;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Predicates;
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.FluentIterable;
|
import com.google.common.collect.FluentIterable;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
@ -2882,7 +2881,7 @@ public class KafkaIndexTaskTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexing.kafka;
|
package org.apache.druid.indexing.kafka;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
@ -99,7 +98,7 @@ public class KafkaRecordSupplierTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -3216,7 +3216,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -29,7 +29,6 @@ import com.amazonaws.services.kinesis.model.Record;
|
||||||
import com.amazonaws.services.kinesis.model.Shard;
|
import com.amazonaws.services.kinesis.model.Shard;
|
||||||
import com.amazonaws.services.kinesis.model.StreamDescription;
|
import com.amazonaws.services.kinesis.model.StreamDescription;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
@ -135,7 +134,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexing.kinesis.supervisor;
|
||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
@ -3582,7 +3581,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
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.ISE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
@ -120,7 +119,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -132,7 +131,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
|
||||||
writeLock.lockInterruptibly();
|
writeLock.lockInterruptibly();
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (entry == null) {
|
if (entry == null) {
|
||||||
|
@ -197,7 +196,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
|
||||||
readLock.lockInterruptibly();
|
readLock.lockInterruptibly();
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (entry == null) {
|
if (entry == null) {
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.guice.annotations.Json;
|
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);
|
return jsonFactory.createParser(input).readValueAs(JacksonUtils.TYPE_REFERENCE_MAP_STRING_STRING);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -216,7 +216,7 @@ public final class CacheScheduler
|
||||||
t.addSuppressed(e);
|
t.addSuppressed(e);
|
||||||
}
|
}
|
||||||
if (Thread.currentThread().isInterrupted() || t instanceof InterruptedException || t instanceof Error) {
|
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) {
|
catch (Exception e) {
|
||||||
log.error(e, "Error emitting namespace stats");
|
log.error(e, "Error emitting namespace stats");
|
||||||
if (Thread.currentThread().isInterrupted()) {
|
if (Thread.currentThread().isInterrupted()) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.server.lookup.namespace.cache;
|
package org.apache.druid.server.lookup.namespace.cache;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import org.apache.druid.java.util.common.concurrent.ExecutorServices;
|
import org.apache.druid.java.util.common.concurrent.ExecutorServices;
|
||||||
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
||||||
|
@ -73,7 +72,7 @@ public abstract class NamespaceExtractionCacheManager
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.error(e, "Error emitting namespace stats");
|
log.error(e, "Error emitting namespace stats");
|
||||||
if (Thread.currentThread().isInterrupted()) {
|
if (Thread.currentThread().isInterrupted()) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -145,7 +145,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
|
||||||
log.info("Using file [%s] for mapDB off heap namespace cache", tmpFile.getAbsolutePath());
|
log.info("Using file [%s] for mapDB off heap namespace cache", tmpFile.getAbsolutePath());
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
mmapDB = DBMaker
|
mmapDB = DBMaker
|
||||||
.newFileDB(tmpFile)
|
.newFileDB(tmpFile)
|
||||||
|
@ -182,7 +182,7 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.query.lookup.namespace;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
@ -80,7 +79,7 @@ public class JSONFlatDataParserTest
|
||||||
return MAPPER.writeValueAsString(input);
|
return MAPPER.writeValueAsString(input);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.server.lookup.namespace;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.data.SearchableVersionedDataFinder;
|
import org.apache.druid.data.SearchableVersionedDataFinder;
|
||||||
|
@ -97,7 +96,7 @@ public class UriCacheGeneratorTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (URISyntaxException e) {
|
catch (URISyntaxException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
return newURI;
|
return newURI;
|
||||||
}
|
}
|
||||||
|
@ -133,7 +132,7 @@ public class UriCacheGeneratorTest
|
||||||
return new FileOutputStream(outFile);
|
return new FileOutputStream(outFile);
|
||||||
}
|
}
|
||||||
catch (IOException ex) {
|
catch (IOException ex) {
|
||||||
throw Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -163,7 +162,7 @@ public class UriCacheGeneratorTest
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
catch (IOException ex) {
|
catch (IOException ex) {
|
||||||
throw Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.server.lookup.namespace.cache;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
|
@ -263,13 +262,13 @@ public class CacheSchedulerTest
|
||||||
testDelete();
|
testDelete();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -333,7 +332,7 @@ public class CacheSchedulerTest
|
||||||
}
|
}
|
||||||
catch (ExecutionException e) {
|
catch (ExecutionException e) {
|
||||||
if (!future.isCancelled()) {
|
if (!future.isCancelled()) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.server.lookup.namespace.cache;
|
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.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
@ -189,7 +188,7 @@ public class JdbcExtractionNamespaceTest
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -237,7 +236,7 @@ public class JdbcExtractionNamespaceTest
|
||||||
lifecycle.start();
|
lifecycle.start();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
closer.register(
|
closer.register(
|
||||||
new Closeable()
|
new Closeable()
|
||||||
|
@ -295,7 +294,7 @@ public class JdbcExtractionNamespaceTest
|
||||||
closer.close();
|
closer.close();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.server.lookup.namespace.cache;
|
package org.apache.druid.server.lookup.namespace.cache;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
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.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
|
@ -92,7 +91,7 @@ public class NamespaceExtractionCacheManagersTest
|
||||||
thunder.await();
|
thunder.await();
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
for (int i = 0; i < 1000; ++i) {
|
for (int i = 0; i < 1000; ++i) {
|
||||||
CacheHandler cacheHandler = manager.createCache();
|
CacheHandler cacheHandler = manager.createCache();
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class S3DataSegmentMover implements DataSegmentMover
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
Throwables.propagateIfInstanceOf(e, AmazonServiceException.class);
|
Throwables.propagateIfInstanceOf(e, AmazonServiceException.class);
|
||||||
Throwables.propagateIfInstanceOf(e, SegmentLoadingException.class);
|
Throwables.propagateIfInstanceOf(e, SegmentLoadingException.class);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.amazonaws.services.s3.model.S3Object;
|
||||||
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -100,7 +99,7 @@ public class S3DataSegmentPuller implements URIDataPuller
|
||||||
throw new IOException("Recoverable exception", e);
|
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);
|
throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.storage.s3;
|
||||||
|
|
||||||
import com.amazonaws.AmazonServiceException;
|
import com.amazonaws.AmazonServiceException;
|
||||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -107,7 +106,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
|
||||||
throw new IOException(e);
|
throw new IOException(e);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
log.info("Deleting temporary cached index.zip");
|
log.info("Deleting temporary cached index.zip");
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class S3TaskLogs implements TaskLogs
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
Throwables.propagateIfInstanceOf(e, IOException.class);
|
Throwables.propagateIfInstanceOf(e, IOException.class);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.storage.s3;
|
package org.apache.druid.storage.s3;
|
||||||
|
|
||||||
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.data.SearchableVersionedDataFinder;
|
import org.apache.druid.data.SearchableVersionedDataFinder;
|
||||||
import org.apache.druid.java.util.common.RetryUtils;
|
import org.apache.druid.java.util.common.RetryUtils;
|
||||||
|
@ -93,7 +92,7 @@ public class S3TimestampVersionedDataFinder extends S3DataSegmentPuller implemen
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.indexer;
|
||||||
|
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.hash.HashFunction;
|
import com.google.common.hash.HashFunction;
|
||||||
|
@ -220,7 +219,7 @@ public class DetermineHashedPartitionsJob implements Jobby
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Function;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Splitter;
|
import com.google.common.base.Splitter;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ComparisonChain;
|
import com.google.common.collect.ComparisonChain;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
@ -272,7 +271,7 @@ public class DeterminePartitionsJob implements Jobby
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Splitter;
|
import com.google.common.base.Splitter;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
|
@ -168,7 +167,7 @@ public class HadoopDruidIndexerConfig
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -182,7 +181,7 @@ public class HadoopDruidIndexerConfig
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -199,7 +198,7 @@ public class HadoopDruidIndexerConfig
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
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));
|
conf.set(HadoopDruidIndexerConfig.CONFIG_PROPERTY, HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(this));
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.indexer;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
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",
|
+ " either there was no input data to process or all the input events were discarded due to some error",
|
||||||
e.getMessage()
|
e.getMessage()
|
||||||
);
|
);
|
||||||
Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
List<DataSegment> publishedSegments = publishedSegmentsBuilder.build();
|
List<DataSegment> publishedSegments = publishedSegmentsBuilder.build();
|
||||||
|
|
||||||
|
@ -727,7 +726,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.error(e, "persist index error");
|
log.error(e, "persist index error");
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
// close this index
|
// close this index
|
||||||
|
@ -856,7 +855,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (ExecutionException | TimeoutException e) {
|
catch (ExecutionException | TimeoutException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
index.close();
|
index.close();
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexer;
|
||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import org.apache.druid.indexer.updater.HadoopDruidConverterConfig;
|
import org.apache.druid.indexer.updater.HadoopDruidConverterConfig;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
|
@ -174,7 +173,7 @@ public class JobHelper
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -344,7 +343,7 @@ public class JobHelper
|
||||||
config.addInputPaths(job);
|
config.addInputPaths(job);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -673,7 +672,7 @@ public class JobHelper
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -858,7 +857,7 @@ public class JobHelper
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.error(e, "Failed to cleanup path[%s]", path);
|
log.error(e, "Failed to cleanup path[%s]", path);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexer.hadoop;
|
package org.apache.druid.indexer.hadoop;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.io.Closeables;
|
import com.google.common.io.Closeables;
|
||||||
|
@ -104,7 +103,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (IOException ex) {
|
catch (IOException ex) {
|
||||||
throw Throwables.propagate(ex);
|
throw new RuntimeException(ex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexer.updater;
|
||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -205,7 +204,7 @@ public class HadoopConverterJob
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (JsonProcessingException e) {
|
catch (JsonProcessingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -320,13 +319,13 @@ public class HadoopConverterJob
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (final IOException e) {
|
catch (final IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
try (final InputStream stream = fs.open(input)) {
|
try (final InputStream stream = fs.open(input)) {
|
||||||
return HadoopDruidConverterConfig.jsonMapper.readValue(stream, DataSegment.class);
|
return HadoopDruidConverterConfig.jsonMapper.readValue(stream, DataSegment.class);
|
||||||
}
|
}
|
||||||
catch (final IOException e) {
|
catch (final IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -344,9 +343,9 @@ public class HadoopConverterJob
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException | ClassNotFoundException e) {
|
catch (InterruptedException | ClassNotFoundException e) {
|
||||||
RuntimeException exception = Throwables.propagate(e);
|
RuntimeException r = new RuntimeException(e);
|
||||||
throwable = exception;
|
throwable = r;
|
||||||
throw exception;
|
throw r;
|
||||||
}
|
}
|
||||||
catch (Throwable t) {
|
catch (Throwable t) {
|
||||||
throwable = t;
|
throwable = t;
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexer;
|
||||||
import com.fasterxml.jackson.databind.InjectableValues;
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
@ -103,7 +102,7 @@ public class BatchDeltaIngestionTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexer;
|
||||||
import com.fasterxml.jackson.databind.InjectableValues;
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Strings;
|
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.HashedPartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
|
@ -71,7 +70,7 @@ public class HadoopIngestionSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) schema.getDataSchema().getGranularitySpec();
|
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) schema.getDataSchema().getGranularitySpec();
|
||||||
|
@ -111,7 +110,7 @@ public class HadoopIngestionSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) schema.getDataSchema().getGranularitySpec();
|
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) schema.getDataSchema().getGranularitySpec();
|
||||||
|
@ -142,7 +141,7 @@ public class HadoopIngestionSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
final PartitionsSpec partitionsSpec = schema.getTuningConfig().getPartitionsSpec();
|
final PartitionsSpec partitionsSpec = schema.getTuningConfig().getPartitionsSpec();
|
||||||
|
@ -188,7 +187,7 @@ public class HadoopIngestionSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
final PartitionsSpec partitionsSpec = schema.getTuningConfig().getPartitionsSpec();
|
final PartitionsSpec partitionsSpec = schema.getTuningConfig().getPartitionsSpec();
|
||||||
|
@ -261,7 +260,7 @@ public class HadoopIngestionSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
@ -326,7 +325,7 @@ public class HadoopIngestionSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
@ -342,7 +341,7 @@ public class HadoopIngestionSpecTest
|
||||||
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
|
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexer;
|
package org.apache.druid.indexer;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
|
@ -89,7 +88,7 @@ public class HadoopTuningConfigTest
|
||||||
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
|
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexer.hadoop;
|
package org.apache.druid.indexer.hadoop;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
@ -83,14 +82,14 @@ public class FSSpideringIteratorTest
|
||||||
Assert.assertTrue(files.isEmpty());
|
Assert.assertTrue(files.isEmpty());
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
try {
|
try {
|
||||||
FileUtils.deleteDirectory(baseDir);
|
FileUtils.deleteDirectory(baseDir);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexer.partitions;
|
package org.apache.druid.indexer.partitions;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -48,7 +47,7 @@ public class HashedPartitionsSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
@ -94,7 +93,7 @@ public class HashedPartitionsSpecTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
@ -136,7 +135,7 @@ public class HashedPartitionsSpecTest
|
||||||
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
|
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.base.Preconditions;
|
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.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
|
@ -91,7 +90,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
|
||||||
return result.isOk() ? result.getTaskLock() : null;
|
return result.isOk() ? result.getTaskLock() : null;
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexing.common.actions;
|
package org.apache.druid.indexing.common.actions;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.discovery.DruidLeaderClient;
|
import org.apache.druid.discovery.DruidLeaderClient;
|
||||||
import org.apache.druid.indexing.common.RetryPolicy;
|
import org.apache.druid.indexing.common.RetryPolicy;
|
||||||
import org.apache.druid.indexing.common.RetryPolicyFactory;
|
import org.apache.druid.indexing.common.RetryPolicyFactory;
|
||||||
|
@ -109,7 +108,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
||||||
Thread.sleep(sleepTime);
|
Thread.sleep(sleepTime);
|
||||||
}
|
}
|
||||||
catch (InterruptedException e2) {
|
catch (InterruptedException e2) {
|
||||||
throw Throwables.propagate(e2);
|
throw new RuntimeException(e2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.actions;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
|
@ -197,7 +196,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
Thread.currentThread().interrupt();
|
Thread.currentThread().interrupt();
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
log.error(
|
log.error(
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
@ -209,7 +208,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.warn(e, "Failed to merge and upload");
|
log.warn(e, "Failed to merge and upload");
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
try {
|
try {
|
||||||
|
@ -250,7 +249,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.warn(e, "Failed to spill index[%d]", indexToPersist.getCount());
|
log.warn(e, "Failed to spill index[%d]", indexToPersist.getCount());
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -425,7 +425,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.inject.Injector;
|
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");
|
return fName.startsWith("druid") && fName.endsWith(".jar") && !fName.endsWith("selfcontained.jar");
|
||||||
}
|
}
|
||||||
catch (URISyntaxException e) {
|
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);
|
return (OutputType) method.invoke(null, input);
|
||||||
}
|
}
|
||||||
catch (IllegalAccessException | InvocationTargetException | ClassNotFoundException | NoSuchMethodException e) {
|
catch (IllegalAccessException | InvocationTargetException | ClassNotFoundException | NoSuchMethodException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
Thread.currentThread().setContextClassLoader(oldLoader);
|
Thread.currentThread().setContextClassLoader(oldLoader);
|
||||||
|
@ -245,7 +244,7 @@ public abstract class HadoopTask extends AbstractTask
|
||||||
return clazz.newInstance();
|
return clazz.newInstance();
|
||||||
}
|
}
|
||||||
catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
|
catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
Thread.currentThread().setContextClassLoader(oldLoader);
|
Thread.currentThread().setContextClassLoader(oldLoader);
|
||||||
|
|
|
@ -1069,7 +1069,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (TimeoutException | ExecutionException e) {
|
catch (TimeoutException | ExecutionException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
@ -86,7 +85,7 @@ public class MergeTask extends MergeTaskBase
|
||||||
return toolbox.getIndexIO().loadIndex(input);
|
return toolbox.getIndexIO().loadIndex(input);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
@ -313,7 +312,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
return lock.getVersion();
|
return lock.getVersion();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -517,7 +516,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.BiMap;
|
import com.google.common.collect.BiMap;
|
||||||
import com.google.common.collect.HashBiMap;
|
import com.google.common.collect.HashBiMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
@ -225,7 +224,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
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);
|
return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
|
||||||
}
|
}
|
||||||
catch (SegmentLoadingException e) {
|
catch (SegmentLoadingException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -27,7 +27,6 @@ import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Splitter;
|
import com.google.common.base.Splitter;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
@ -474,7 +473,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
||||||
}
|
}
|
||||||
catch (Throwable t) {
|
catch (Throwable t) {
|
||||||
log.info(t, "Exception caught during execution");
|
log.info(t, "Exception caught during execution");
|
||||||
throw Throwables.propagate(t);
|
throw new RuntimeException(t);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
@ -146,7 +145,7 @@ public class MetadataTaskStorage implements TaskStorage
|
||||||
if (e instanceof EntryExistsException) {
|
if (e instanceof EntryExistsException) {
|
||||||
throw (EntryExistsException) e;
|
throw (EntryExistsException) e;
|
||||||
} else {
|
} else {
|
||||||
Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -340,7 +340,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
lifecycleLock.started();
|
lifecycleLock.started();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycleLock.exitStart();
|
lifecycleLock.exitStart();
|
||||||
|
@ -378,7 +378,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycleLock.exitStop();
|
lifecycleLock.exitStop();
|
||||||
|
@ -617,12 +617,12 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
).get();
|
).get();
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (ExecutionException e) {
|
catch (ExecutionException e) {
|
||||||
// Unwrap if possible
|
// Unwrap if possible
|
||||||
Throwables.propagateIfPossible(e.getCause(), IOException.class);
|
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);
|
log.info("Tried to delete status path[%s] that didn't exist! Must've gone away already?", statusPath);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1047,7 +1047,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
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));
|
scheduleTasksCleanupForWorker(worker.getHost(), getAssignedTasks(worker));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
try {
|
try {
|
||||||
|
@ -1149,7 +1149,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert("Exception while cleaning up worker[%s]", worker).emit();
|
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) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return getWorkerFromZK(lazyWorkers.values());
|
return getWorkerFromZK(lazyWorkers.values());
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexing.overlord;
|
package org.apache.druid.indexing.overlord;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.client.indexing.IndexingService;
|
import org.apache.druid.client.indexing.IndexingService;
|
||||||
import org.apache.druid.curator.discovery.ServiceAnnouncer;
|
import org.apache.druid.curator.discovery.ServiceAnnouncer;
|
||||||
|
@ -152,7 +151,7 @@ public class TaskMaster implements TaskCountStatsProvider
|
||||||
leaderLifecycle.start();
|
leaderLifecycle.start();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
giant.unlock();
|
giant.unlock();
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexing.overlord;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -585,7 +584,7 @@ public class TaskQueue
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.warn(e, "Failed to sync tasks from storage!");
|
log.warn(e, "Failed to sync tasks from storage!");
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
giant.unlock();
|
giant.unlock();
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.indexing.overlord;
|
package org.apache.druid.indexing.overlord;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.indexer.TaskLocation;
|
import org.apache.druid.indexer.TaskLocation;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexing.worker.Worker;
|
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();
|
return new URI(StringUtils.format("%s://%s%s", worker.getScheme(), worker.getHost(), path)).toURL();
|
||||||
}
|
}
|
||||||
catch (URISyntaxException | MalformedURLException e) {
|
catch (URISyntaxException | MalformedURLException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import org.apache.curator.framework.recipes.cache.ChildData;
|
import org.apache.curator.framework.recipes.cache.ChildData;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
|
@ -70,7 +69,7 @@ public class ZkWorker implements Closeable
|
||||||
return jsonMapper.readValue(input.getData(), TaskAnnouncement.class);
|
return jsonMapper.readValue(input.getData(), TaskAnnouncement.class);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import org.apache.druid.guice.annotations.Json;
|
import org.apache.druid.guice.annotations.Json;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
|
||||||
|
@ -81,7 +80,7 @@ public class GalaxyEC2UserData implements EC2UserData<GalaxyEC2UserData>
|
||||||
return StringUtils.encodeBase64String(jsonMapper.writeValueAsBytes(this));
|
return StringUtils.encodeBase64String(jsonMapper.writeValueAsBytes(this));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -248,7 +248,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
log.info("Started.");
|
log.info("Started.");
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycleLock.exitStart();
|
lifecycleLock.exitStart();
|
||||||
|
@ -539,7 +539,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
scheduleTasksCleanupForWorker(worker.getHost());
|
scheduleTasksCleanupForWorker(worker.getHost());
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
checkAndRemoveWorkersFromBlackList();
|
checkAndRemoveWorkersFromBlackList();
|
||||||
|
@ -597,7 +597,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert("Exception while cleaning up worker[%s]", workerHostAndPort).emit();
|
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) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return getLazyWorkers();
|
return getLazyWorkers();
|
||||||
|
@ -871,12 +871,12 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
|
||||||
).get();
|
).get();
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
catch (ExecutionException e) {
|
catch (ExecutionException e) {
|
||||||
// Unwrap if possible
|
// Unwrap if possible
|
||||||
Throwables.propagateIfPossible(e.getCause(), IOException.class);
|
Throwables.propagateIfPossible(e.getCause(), IOException.class);
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.indexing.overlord.http;
|
package org.apache.druid.indexing.overlord.http;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.indexing.overlord.TaskMaster;
|
import org.apache.druid.indexing.overlord.TaskMaster;
|
||||||
|
@ -70,7 +69,7 @@ public class OverlordRedirectInfo implements RedirectInfo
|
||||||
return new URL(location);
|
return new URL(location);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
|
import org.apache.druid.indexing.overlord.ImmutableWorkerInfo;
|
||||||
|
@ -68,7 +67,7 @@ public class JavaScriptWorkerSelectStrategy implements WorkerSelectStrategy
|
||||||
return ((Invocable) engine).getInterface(SelectorFunction.class);
|
return ((Invocable) engine).getInterface(SelectorFunction.class);
|
||||||
}
|
}
|
||||||
catch (ScriptException e) {
|
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
Loading…
Reference in New Issue