Skip to content

Commit

Permalink
[FLINK-4032] Replace all usage of Guava Preconditions
Browse files Browse the repository at this point in the history
This closes apache#2084
  • Loading branch information
zentol committed Jun 15, 2016
1 parent 279777f commit c4bbed9
Show file tree
Hide file tree
Showing 152 changed files with 240 additions and 229 deletions.
17 changes: 17 additions & 0 deletions flink-core/src/main/java/org/apache/flink/util/Preconditions.java
Original file line number Diff line number Diff line change
Expand Up @@ -234,6 +234,23 @@ public static void checkElementIndex(int index, int size) {
}
}

/**
* Ensures that the given index is valid for an array, list or string of the given size.
*
* @param index index to check
* @param size size of the array, list or string
* @param errorMessage The message for the {@code IndexOutOfBoundsException} that is thrown if the check fails.
*
* @throws IllegalArgumentException Thrown, if size is negative.
* @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to size
*/
public static void checkElementIndex(int index, int size, @Nullable String errorMessage) {
checkArgument(size >= 0, "Size was negative.");
if (index < 0 || index >= size) {
throw new IndexOutOfBoundsException(String.valueOf(errorMessage) + " Index: " + index + ", Size: " + size);
}
}

// ------------------------------------------------------------------------
// Utilities
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,12 @@
*/
package org.apache.flink.api.table.runtime.aggregate

import com.google.common.base.Preconditions
import org.apache.flink.api.common.functions.RichMapFunction
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.ResultTypeQueryable
import org.apache.flink.api.table.Row
import org.apache.flink.configuration.Configuration
import org.apache.flink.util.Preconditions

class AggregateMapFunction[IN, OUT](
private val aggregates: Array[Aggregate[_]],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,11 @@ package org.apache.flink.api.table.runtime.aggregate

import java.lang.Iterable

import com.google.common.base.Preconditions
import org.apache.flink.api.common.functions.{CombineFunction, RichGroupReduceFunction, RichMapPartitionFunction}
import org.apache.flink.api.table.Row
import org.apache.flink.configuration.Configuration
import org.apache.flink.util.Collector
import org.apache.flink.util.Preconditions

import scala.collection.JavaConversions._

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.LocalStrategy;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A Channel represents the result produced by an operator and the data exchange
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Back pressure statistics tracker.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@

import java.util.WeakHashMap;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Gateway to obtaining an {@link ExecutionGraph} from a source, like JobManager or Archive.
Expand Down Expand Up @@ -95,4 +95,4 @@ else if (result instanceof JobManagerMessages.JobFound) {
throw new RuntimeException("Error requesting execution graph", e);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
import java.util.UUID;
import java.util.concurrent.TimeoutException;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Retrieves and stores the actor gateway to the current leading JobManager. In case of an error,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@

import java.util.Arrays;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Back pressure statistics of multiple tasks.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@
import java.util.HashMap;
import java.util.Map;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* The Netty channel handler that processes all HTTP requests.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* The Netty channel handler that processes all HTTP requests.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import java.util.List;
import java.util.Map;

import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkArgument;

/**
* A sample of stack traces for one or more tasks.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,8 @@
import java.util.TimerTask;
import java.util.UUID;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A coordinator for triggering and collecting stack traces of running tasks.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* The root component of the web runtime monitor. This class starts the web server and creates
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@
import java.util.Locale;
import java.util.TimeZone;

import static com.google.common.base.Preconditions.checkNotNull;
import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
Expand All @@ -85,6 +84,7 @@
import static io.netty.handler.codec.http.HttpResponseStatus.NOT_MODIFIED;
import static io.netty.handler.codec.http.HttpResponseStatus.OK;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Simple file server handler that serves requests to web frontend's static files, such as
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import java.io.StringWriter;
import java.util.Map;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Responder that returns the status of the Flink cluster, such as how many
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import java.io.StringWriter;
import java.util.Map;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Request handler that returns a summary of the job status.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Utilities to extract a redirect address.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,8 @@
import java.io.StringWriter;
import java.util.Map;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Request handler that returns back pressure stats for a single job vertex and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,11 +71,11 @@
import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;

import static com.google.common.base.Preconditions.checkNotNull;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
import static io.netty.handler.codec.http.HttpResponseStatus.OK;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Request handler that returns the TaskManager log/out files.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* This class implements the BLOB server. The BLOB server is responsible for listening for incoming requests and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
import java.security.NoSuchAlgorithmException;
import java.util.UUID;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Utility class to work with blob data.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import java.io.OutputStream;
import java.net.URI;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Blob store backed by {@link FileSystem}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,10 @@
import org.apache.flink.runtime.operators.BatchTask;
import org.apache.flink.runtime.operators.util.ReaderIterator;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.google.common.base.Preconditions;

/**
* @param <T> The type of the elements in the broadcasted data set.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,8 @@
import java.util.TimerTask;
import java.util.UUID;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* The checkpoint coordinator coordinates the distributed snapshots of operators and state.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,10 @@

package org.apache.flink.runtime.checkpoint;

import com.google.common.base.Preconditions;
import org.apache.flink.runtime.akka.FlinkUntypedActor;
import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.messages.ExecutionGraphMessages;
import org.apache.flink.util.Preconditions;

import java.util.UUID;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import java.io.ObjectOutputStream;
import java.io.Serializable;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* {@link FileSystem} backed {@link StateStore}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Java heap backed {@link StateStore}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* The savepoint coordinator is a slightly modified variant of the regular
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,10 @@

package org.apache.flink.runtime.checkpoint;

import com.google.common.base.Preconditions;
import org.apache.flink.runtime.akka.FlinkUntypedActor;
import org.apache.flink.runtime.jobgraph.JobStatus;
import org.apache.flink.runtime.messages.ExecutionGraphMessages;
import org.apache.flink.util.Preconditions;

import java.util.UUID;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
import java.util.ArrayList;
import java.util.List;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* {@link CompletedCheckpointStore} for JobManagers running in {@link RecoveryMode#STANDALONE}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@

import java.io.Serializable;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Simple bean to describe the state belonging to a parallel operator. It is part of the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* {@link CheckpointIDCounter} instances for JobManagers running in {@link RecoveryMode#ZOOKEEPER}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.flink.runtime.jobmanager.RecoveryMode;
import org.apache.flink.runtime.util.ZooKeeperUtils;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* {@link CheckpointCoordinator} components in {@link RecoveryMode#ZOOKEEPER}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
import java.util.ConcurrentModificationException;
import java.util.List;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* {@link CompletedCheckpointStore} for JobManagers running in {@link RecoveryMode#ZOOKEEPER}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import java.util.Arrays;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* Statistics for a specific checkpoint per operator.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import java.util.List;
import java.util.Map;

import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkArgument;

/**
* A simple checkpoint stats tracker.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@
import java.net.InetSocketAddress;
import java.util.concurrent.TimeoutException;

import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* The JobClient bridges between the JobManager's asynchronous actor messages and
Expand Down
Loading

0 comments on commit c4bbed9

Please sign in to comment.