Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@
import com.google.api.services.pubsub.model.Subscription;
import com.google.api.services.pubsub.model.Topic;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Uninterruptibles;
Expand Down Expand Up @@ -116,7 +115,7 @@ public void setup() throws IOException {
Thread.currentThread().interrupt();
// Ignore InterruptedException
}
Throwables.propagate(lastException);
throw new RuntimeException(lastException);
}

/**
Expand Down
14 changes: 0 additions & 14 deletions runners/direct-java/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -278,20 +278,6 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client</artifactId>
<version>${google-clients.version}</version>
<exclusions>
<!-- Exclude an old version of guava that is being pulled
in by a transitive dependency of google-api-client -->
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>guava-jdk5</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client-protobuf</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.PCollection;

import com.google.api.client.util.Throwables;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.SetMultimap;

Expand Down Expand Up @@ -100,7 +99,7 @@ public UncommittedBundle<T> add(WindowedValue<T> element) {
mutationDetectors.put(
element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
} catch (CoderException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
underlying.add(element);
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.PValue;

import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
Expand Down Expand Up @@ -256,7 +255,10 @@ public InProcessPipelineResult run(Pipeline pipeline) {
} catch (UserCodeException userException) {
throw new PipelineExecutionException(userException.getCause());
} catch (Throwable t) {
Throwables.propagate(t);
if (t instanceof RuntimeException) {
throw (RuntimeException) t;
}
throw new RuntimeException(t);
}
}
return result;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.apache.beam.sdk.values.PCollectionView;

import com.google.common.base.MoreObjects;
import com.google.common.base.Throwables;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
Expand Down Expand Up @@ -172,7 +171,7 @@ private void updatePCollectionViewWindowValues(
future.set(Collections.<WindowedValue<?>>emptyList());
}
} catch (ExecutionException e) {
Throwables.propagate(e.getCause());
throw new RuntimeException(e.getCause());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,6 @@
import org.apache.beam.sdk.transforms.AppliedPTransform;
import org.apache.beam.sdk.util.WindowedValue;

import com.google.common.base.Throwables;

import java.util.ArrayList;
import java.util.Collection;
import java.util.concurrent.Callable;
Expand Down Expand Up @@ -119,7 +117,10 @@ public InProcessTransformResult call() {
return result;
} catch (Throwable t) {
onComplete.handleThrowable(inputBundle, t);
throw Throwables.propagate(t);
if (t instanceof RuntimeException) {
throw (RuntimeException) t;
}
throw new RuntimeException(t);
} finally {
transformEvaluationState.complete(this);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@
import com.google.api.services.dataflow.model.JobMetrics;
import com.google.api.services.dataflow.model.MetricUpdate;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -356,7 +355,7 @@ private boolean nextBackOff(Sleeper sleeper, BackOff backoff) {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

Expand Down
6 changes: 6 additions & 0 deletions sdks/java/build-tools/src/main/resources/beam/checkstyle.xml
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,12 @@ page at http://checkstyle.sourceforge.net/config.html -->
<property name="severity" value="error"/>
</module>

<module name="RegexpSingleline">
<property name="format" value="Throwables.propagate\("/>
<property name="message" value="Throwables.propagate is deprecated"/>
<property name="severity" value="error"/>
</module>

<!-- Allow use of comment to suppress javadocstyle -->
<module name="SuppressionCommentFilter">
<property name="offCommentFormat" value="CHECKSTYLE.OFF\: ([\w\|]+)"/>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@
import com.google.api.services.pubsub.model.ReceivedMessage;
import com.google.api.services.pubsub.model.Subscription;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;

import org.joda.time.Duration;
Expand Down Expand Up @@ -814,7 +813,7 @@ public void processElement(ProcessContext c) throws IOException {
}
}
if (finallyBlockException != null) {
Throwables.propagate(finallyBlockException);
throw new RuntimeException(finallyBlockException);
}

for (PubsubMessage message : messages) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,14 @@
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.util.StringUtils;
import org.apache.beam.sdk.util.common.ReflectHelpers;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.Collections2;
import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableListMultimap;
Expand Down Expand Up @@ -991,7 +991,7 @@ private static List<PropertyDescriptor> validateClass(Class<? extends PipelineOp
methods.add(klass.getMethod("cloneAs", Class.class));
methods.add(klass.getMethod("populateDisplayData", DisplayData.Builder.class));
} catch (NoSuchMethodException | SecurityException e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}

// Verify that there are no methods with the same name with two different return types.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableMap;
import com.google.common.reflect.TypeParameter;
Expand Down Expand Up @@ -479,7 +478,7 @@ private <InputT, OutputT> void invoke(Method m,
throw UserCodeException.wrap(e.getCause());
} catch (IllegalAccessException | IllegalArgumentException e) {
// Exception in our code.
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ public void processElement(final ProcessContext c) throws Exception {
}

if (failure.get() != null) {
throw Throwables.propagate(failure.get());
throw new RuntimeException(failure.get());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also replace the Throwables.propagateIfPossible on line 234?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Prefer to deprecate and replace different functions in different commits as
for sanity

On Mon, May 2, 2016 at 1:44 PM, Thomas Groh notifications@github.com
wrote:

In
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java
#270 (comment):

@@ -221,7 +221,7 @@ public void processElement(final ProcessContext c) throws Exception {
}

   if (failure.get() != null) {
  •    throw Throwables.propagate(failure.get());
    

Also replace the Throwables.propagateIfPossible on line 234?


You are receiving this because you authored the thread.
Reply to this email directly or view it on GitHub
https://github.com/apache/incubator-beam/pull/270/files/046feca39256399c891e5ab386a67f1a8cd8deee#r61800893

}

executor.submit(new Runnable() {
Expand All @@ -246,7 +246,7 @@ public void finishBundle(Context c) throws Exception {
// processElement calls have finished.
workTickets.acquire(maxParallelism);
if (failure.get() != null) {
throw Throwables.propagate(failure.get());
throw new RuntimeException(failure.get());
}
doFn.finishBundle(c);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@
import com.google.cloud.hadoop.util.ApiErrorExtractor;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.MoreExecutors;

import org.slf4j.Logger;
Expand Down Expand Up @@ -275,7 +274,7 @@ public List<TableDataInsertAllResponse.InsertErrors> call() throws IOException {
Thread.currentThread().interrupt();
throw new IOException("Interrupted while inserting " + rowsToPublish);
} catch (ExecutionException e) {
Throwables.propagate(e.getCause());
throw new RuntimeException(e.getCause());
}

if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,6 @@
import com.google.common.base.Function;
import com.google.common.base.MoreObjects;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
Expand Down Expand Up @@ -408,7 +407,7 @@ public WindowedValue<InputT> apply(TimestampedValue<InputT> input) {
windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE)));
return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING);
} catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}
}));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;

import static org.junit.Assert.assertTrue;

import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
Expand All @@ -45,7 +44,6 @@

import com.google.common.base.MoreObjects;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;

Expand Down Expand Up @@ -257,7 +255,7 @@ public final void injectElements(Collection<TimestampedValue<InputT>> values) th

windowedValues.add(WindowedValue.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING));
} catch (Exception e) {
throw Throwables.propagate(e);
throw new RuntimeException(e);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -634,7 +633,7 @@ public UnboundedKafkaReader<K, V> createReader(PipelineOptions options,
return new UnboundedKafkaReader<K, V>(
generateInitialSplits(1, options).get(0), checkpointMark);
} catch (Exception e) {
Throwables.propagate(e);
throw new RuntimeException(e);
}
}
return new UnboundedKafkaReader<K, V>(this, checkpointMark);
Expand Down