提交 80d1c06d 编写于 作者: A Aljoscha Krettek

[FLINK-6107] Custom checkstyle for flink-streaming-java

This is based on the Apache Beam checkstyle with some custom
modifications for making it fit the exisiting Flink code base. The most
notable change is that Flink uses Tabs for indentation while Beam uses
spaces.

This adds a lot of checks that are commented out because they require
non-trivial changes to the code base.

There are some trivial code changes for good practices checkstyle rules
that where only broken in very few places.
上级 70183647
......@@ -123,6 +123,41 @@ under the License.
</execution>
</executions>
</plugin>
<!-- We have more strict checkstyle rules than the rest of the project -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
<version>2.17</version>
<dependencies>
<dependency>
<groupId>com.puppycrawl.tools</groupId>
<artifactId>checkstyle</artifactId>
<version>6.19</version>
</dependency>
</dependencies>
<configuration>
<configLocation>../tools/maven/strict-checkstyle.xml</configLocation>
<suppressionsLocation>../tools/maven/suppressions.xml</suppressionsLocation>
<logViolationsToConsole>true</logViolationsToConsole>
<failOnViolation>true</failOnViolation>
</configuration>
<executions>
<!--
Execute checkstyle after compilation but before tests.
This ensures that any parsing or type checking errors are from
javac, so they look as expected. Beyond that, we want to
fail as early as possible.
-->
<execution>
<phase>test-compile</phase>
<goals>
<goal>check</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
......
......@@ -42,5 +42,5 @@ public interface OutputSelector<OUT> extends Serializable {
* @param value
* Output object for which the output selection should be made.
*/
public Iterable<String> select(OUT value);
Iterable<String> select(OUT value);
}
......@@ -1147,7 +1147,7 @@ public class DataStream<T> {
// configure the type if needed
if (sinkFunction instanceof InputTypeConfigurable) {
((InputTypeConfigurable) sinkFunction).setInputType(getType(), getExecutionConfig() );
((InputTypeConfigurable) sinkFunction).setInputType(getType(), getExecutionConfig());
}
StreamSink<T> sinkOperator = new StreamSink<>(clean(sinkFunction));
......
......@@ -367,7 +367,7 @@ public class ContinuousFileReaderOperator<OUT> extends AbstractStreamOperator<OU
private List<TimestampedFileInputSplit> getReaderState() throws IOException {
List<TimestampedFileInputSplit> snapshot = new ArrayList<>(this.pendingSplits.size());
if (currentSplit != null ) {
if (currentSplit != null) {
if (this.format instanceof CheckpointableInputFormat && this.isSplitOpen) {
Serializable formatState =
((CheckpointableInputFormat<TimestampedFileInputSplit, Serializable>) this.format).getCurrentState();
......
......@@ -178,9 +178,9 @@ public abstract class MessageAcknowledgingSourceBase<Type, UId>
/**
* This method must be implemented to acknowledge the given set of IDs back to the message queue.
* @param UIds The list od IDs to acknowledge.
* @param uIds The list od IDs to acknowledge.
*/
protected abstract void acknowledgeIDs(long checkpointId, List<UId> UIds);
protected abstract void acknowledgeIDs(long checkpointId, List<UId> uIds);
/**
* Adds an ID to be stored with the current checkpoint.
......
......@@ -42,6 +42,6 @@ public interface DeltaFunction<DATA> extends Serializable {
* the new data point.
* @return the delta between the two given points.
*/
public double getDelta(DATA oldDataPoint, DATA newDataPoint);
double getDelta(DATA oldDataPoint, DATA newDataPoint);
}
......@@ -41,6 +41,6 @@ public interface Extractor<FROM, TO> extends Serializable {
* the input data
* @return the extracted/converted data
*/
public TO extract(FROM in);
TO extract(FROM in);
}
......@@ -34,15 +34,15 @@ public class StreamEdge implements Serializable {
private static final long serialVersionUID = 1L;
final private String edgeId;
private final String edgeId;
final private StreamNode sourceVertex;
final private StreamNode targetVertex;
private final StreamNode sourceVertex;
private final StreamNode targetVertex;
/**
* The type number of the input for co-tasks.
*/
final private int typeNumber;
private final int typeNumber;
/**
* A list of output names that the target vertex listens to (if there is
......
......@@ -369,7 +369,7 @@ public class StreamGraphGenerator {
iterate.getParallelism(),
iterate.getMaxParallelism(),
iterate.getMinResources(),
iterate.getPreferredResources() );
iterate.getPreferredResources());
StreamNode itSource = itSourceAndSink.f0;
StreamNode itSink = itSourceAndSink.f1;
......
......@@ -40,7 +40,7 @@ public class StreamNode implements Serializable {
private static final long serialVersionUID = 1L;
transient private StreamExecutionEnvironment env;
private transient StreamExecutionEnvironment env;
private final int id;
private Integer parallelism = null;
......
......@@ -336,7 +336,7 @@ public class StreamSourceContexts {
* toggles the status. ACTIVE status resumes as soon as some record or watermark is collected
* again.
*/
private static abstract class WatermarkContext<T> implements SourceFunction.SourceContext<T> {
private abstract static class WatermarkContext<T> implements SourceFunction.SourceContext<T> {
protected final ProcessingTimeService timeService;
protected final Object checkpointLock;
......
......@@ -36,5 +36,5 @@ public interface Triggerable<K, N> {
/**
* Invoked when a processing-time timer fires.
*/
void onProcessingTime(InternalTimer<K, N> timer) throws Exception ;
void onProcessingTime(InternalTimer<K, N> timer) throws Exception;
}
......@@ -606,7 +606,7 @@ public class KeyMap<K, V> implements Iterable<KeyMap.Entry<K, V>> {
*
* @param <V> The type created by the factory.
*/
public static interface LazyFactory<V> {
public interface LazyFactory<V> {
/**
* The factory method; creates the value.
......@@ -625,7 +625,7 @@ public class KeyMap<K, V> implements Iterable<KeyMap.Entry<K, V>> {
* @param <K> The type of the key.
* @param <V> The type of the value.
*/
public static interface TraversalEvaluator<K, V> {
public interface TraversalEvaluator<K, V> {
/**
* Called whenever the traversal starts with a new key.
......
......@@ -310,7 +310,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
final TupleSerializer<Tuple2<W, W>> tupleSerializer = new TupleSerializer<>(
typedTuple,
new TypeSerializer[] {windowSerializer, windowSerializer} );
new TypeSerializer[] {windowSerializer, windowSerializer});
final ListStateDescriptor<Tuple2<W, W>> mergingSetsStateDescriptor =
new ListStateDescriptor<>("merging-window-set", tupleSerializer);
......
......@@ -36,5 +36,5 @@ public interface ProcessingTimeCallback {
*
* @param timestamp The timestamp for which the trigger event was scheduled.
*/
void onProcessingTime(long timestamp) throws Exception ;
void onProcessingTime(long timestamp) throws Exception;
}
......@@ -92,7 +92,7 @@ public abstract class FieldAccessor<T, F> implements Serializable {
* This is when the entire record is considered as a single field. (eg. field 0 of a basic type, or a
* field of a POJO that is itself some composite type but is not further decomposed)
*/
final static class SimpleFieldAccessor<T> extends FieldAccessor<T, T> {
static final class SimpleFieldAccessor<T> extends FieldAccessor<T, T> {
private static final long serialVersionUID = 1L;
......@@ -113,7 +113,7 @@ public abstract class FieldAccessor<T, F> implements Serializable {
}
}
final static class ArrayFieldAccessor<T, F> extends FieldAccessor<T, F> {
static final class ArrayFieldAccessor<T, F> extends FieldAccessor<T, F> {
private static final long serialVersionUID = 1L;
......
......@@ -198,9 +198,9 @@ public class FieldAccessorFactory implements Serializable {
// --------------------------------------------------------------------------------------------------
private final static String REGEX_FIELD = "[\\p{L}\\p{Digit}_\\$]*"; // This can start with a digit (because of Tuples)
private final static String REGEX_NESTED_FIELDS = "("+REGEX_FIELD+")(\\.(.+))?";
private final static String REGEX_NESTED_FIELDS_WILDCARD = REGEX_NESTED_FIELDS
private static final String REGEX_FIELD = "[\\p{L}\\p{Digit}_\\$]*"; // This can start with a digit (because of Tuples)
private static final String REGEX_NESTED_FIELDS = "("+REGEX_FIELD+")(\\.(.+))?";
private static final String REGEX_NESTED_FIELDS_WILDCARD = REGEX_NESTED_FIELDS
+"|\\"+ Keys.ExpressionKeys.SELECT_ALL_CHAR
+"|\\"+ Keys.ExpressionKeys.SELECT_ALL_CHAR_SCALA;
......
此差异已折叠。
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册