Skip to content

Commit

Permalink
[FLINK-2775] [CLEANUP] Cleanup code as part of theme to be more consi…
Browse files Browse the repository at this point in the history
…stent on Utils classes

As part of continuing theme and effort to help make the code more consistent, adding cleanup to Utils classes:
-) Add final class modifier to the XXXUtils and XXXUtil classes to make sure can not be extended.
-) Add missing Javadoc header classs to some public classes.
-) Add private constructor to Utils classes to avoid instantiation.
-) Remove unused test/java/org/apache/flink/test/recordJobs/util/ConfigUtils.java class

Author: hsaputra <[email protected]>

Closes apache#1189 from hsaputra/add_missing_javadocs_class and squashes the following commits:

0dfba19 [hsaputra] Cleanup code as part of theme to be more consistent: -) Add final class modifier to the XXXUtils and XXXUtil classes to make sure could not be extended. -) Add missing Javadoc header classs to some public classes. -) Add private constructor to Utils classes to avoid instantiation. -) Remove unused test/java/org/apache/flink/test/recordJobs/util/ConfigUtils.java class
  • Loading branch information
hsaputra committed Sep 29, 2015
1 parent 2598709 commit 136347d
Show file tree
Hide file tree
Showing 33 changed files with 287 additions and 112 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.runtime.net.NetUtils;

public class DataStreamUtils {
public final class DataStreamUtils {

/**
* Returns an iterator to iterate over the elements of the DataStream.
Expand Down Expand Up @@ -83,4 +83,11 @@ public void run(){
}
}
}

/**
* Private constructor to prevent instantiation.
*/
private DataStreamUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,10 @@
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.configuration.Configuration;

public class FunctionUtils {
/**
* Utility class that contains helper methods to work with Flink {@link Function} class.
*/
public final class FunctionUtils {

public static void openFunction(Function function, Configuration parameters) throws Exception{
if (function instanceof RichFunction) {
Expand Down Expand Up @@ -124,4 +127,11 @@ public static Method checkAndExtractLambdaMethod(Function function) {
throw new RuntimeException("Could not extract lambda method out of function: " + e.getClass().getSimpleName() + " - " + e.getMessage(), e);
}
}

/**
* Private constructor to prevent instantiation.
*/
private FunctionUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
/**
* convert the java.lang type into a value type
*/
public class ValueUtil {
public final class ValueUtil {
public static Value toFlinkValueType(Object java) {
if (java == null) {
return NullValue.getInstance();
Expand Down Expand Up @@ -56,4 +56,11 @@ public static Value toFlinkValueType(Object java) {
}
throw new IllegalArgumentException("unsupported Java value");
}

/**
* Private constructor to prevent instantiation.
*/
private ValueUtil() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import java.io.PrintWriter;
import java.io.StringWriter;

public class ExceptionUtils {
public final class ExceptionUtils {

/**
* Makes a string representation of the exception's stack trace, or "(null)", if the
Expand Down Expand Up @@ -113,4 +113,11 @@ else if (t instanceof Error) {
throw (Error) t;
}
}

/**
* Private constructor to prevent instantiation.
*/
private ExceptionUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
/**
* Utility class to create instances from class objects and checking failure reasons.
*/
public class InstantiationUtil {
public final class InstantiationUtil {

/**
* A custom ObjectInputStream that can also load user-code using a
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import java.lang.reflect.ParameterizedType;
import java.lang.reflect.Type;

public class ReflectionUtil {
public final class ReflectionUtil {
public static <T> T newInstance(Class<T> clazz) {
try {
return clazz.newInstance();
Expand Down Expand Up @@ -138,4 +138,11 @@ public static Class<?>[] getTemplateTypes(Class<?> clazz) {
}
return types;
}

/**
* Private constructor to prevent instantiation.
*/
private ReflectionUtil() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@
import org.apache.flink.configuration.GlobalConfiguration;

/**
*
* Utility class to help test the Flink configuration.
*/
public class TestConfigUtils {
public final class TestConfigUtils {

public static final void loadGlobalConf(String[] keys, String[] values) throws IOException {
public static void loadGlobalConf(String[] keys, String[] values) throws IOException {
loadGlobalConf(getConfAsString(keys, values));
}

Expand Down Expand Up @@ -62,7 +62,7 @@ public static void loadGlobalConf(String contents) throws IOException {
}
}

public static final String getConfAsString(String[] keys, String[] values) {
public static String getConfAsString(String[] keys, String[] values) {
if (keys == null || values == null || keys.length != values.length) {
throw new IllegalArgumentException();
}
Expand Down
13 changes: 11 additions & 2 deletions flink-java/src/main/java/org/apache/flink/api/java/Utils.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,10 @@
import org.apache.flink.util.Collector;
import static org.apache.flink.api.java.functions.FunctionAnnotation.SkipCodeAnalysis;


public class Utils {
/**
* Utility class that contains helper methods to work with Java APIs.
*/
public final class Utils {

public static final Random RNG = new Random();

Expand Down Expand Up @@ -177,4 +179,11 @@ private static String getGenericTypeTree(Class type, int indent) {
}
return ret;
}

/**
* Private constructor to prevent instantiation.
*/
private Utils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,10 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;

public class SemanticPropUtil {
/**
* Utility class that contains helper methods to work with {@link SemanticProperties}.
*/
public final class SemanticPropUtil {

private final static String REGEX_WILDCARD = "[\\"+ Keys.ExpressionKeys.SELECT_ALL_CHAR+"\\"+ Keys.ExpressionKeys.SELECT_ALL_CHAR_SCALA+"]";
private final static String REGEX_SINGLE_FIELD = "[\\p{L}\\p{Digit}_\\$]+";
Expand Down Expand Up @@ -698,4 +701,10 @@ private static List<FlatFieldDescriptor> getFlatFields(String fieldStr, TypeInfo
}
}

/**
* Private constructor to prevent instantiation.
*/
private SemanticPropUtil() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;


public class HadoopUtils {
/**
* Utility class to work with Apache Hadoop MapRed classes.
*/
public final class HadoopUtils {

private static final Logger LOG = LoggerFactory.getLogger(HadoopUtils.class);

Expand Down Expand Up @@ -151,4 +153,11 @@ public static org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
}
return retConf;
}

/**
* Private constructor to prevent instantiation.
*/
private HadoopUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,10 @@
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;

public class HadoopUtils {
/**
* Utility class to work with next generatio of Apache Hadoop MapReduce classes.
*/
public final class HadoopUtils {

/**
* Merge HadoopConfiguration into Configuration. This is necessary for the HDFS configuration.
Expand Down Expand Up @@ -79,4 +82,11 @@ public static TaskAttemptContext instantiateTaskAttemptContext(Configuration con
throw new Exception("Could not create instance of TaskAttemptContext.");
}
}

/**
* Private constructor to prevent instantiation.
*/
private HadoopUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public abstract class UdfOperatorUtils {
/**
* Utility class that contains helper methods to work with UDF operators.
*/
public final class UdfOperatorUtils {

private static final Logger LOG = LoggerFactory.getLogger(UdfOperatorUtils.class);

Expand Down Expand Up @@ -100,4 +103,11 @@ else if (mode == CodeAnalysisMode.HINT) {
}
}
}

/**
* Private constructor to prevent instantiation.
*/
private UdfOperatorUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,9 @@
import java.util.Map;
import java.util.Set;

/**
* Utility class to work with {@link UdfAnalyzer}
*/
public final class UdfAnalyzerUtils {

public static TaggedValue convertTypeInfoToTaggedValue(TaggedValue.Input input, TypeInformation<?> typeInfo,
Expand Down Expand Up @@ -339,4 +342,11 @@ else if (value.canContainFields()) {
}
return null;
}

/**
* Private constructor to prevent instantiation.
*/
private UdfAnalyzerUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,9 @@
import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeinfo.TypeInformation;

/**
* Represent a field definition for {@link PojoTypeInfo} type of objects.
*/
public class PojoField implements Serializable {

private static final long serialVersionUID = 1975295846436559363L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@
* This class provides simple utility methods for zipping elements in a data set with an index
* or with a unique identifier.
*/
public class DataSetUtils {
public final class DataSetUtils {

/**
* Method that goes over all the elements in each partition in order to retrieve
Expand Down Expand Up @@ -260,4 +260,11 @@ public static int getBitSize(long value){
return 32 - Integer.numberOfLeadingZeros((int)value);
}
}

/**
* Private constructor to prevent instantiation.
*/
private DataSetUtils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,11 @@


/**
*
* Utility class that contains helper methods for optimizer.
*/
public class Utils
{
public static final FieldList createOrderedFromSet(FieldSet set) {
public final class Utils {

public static FieldList createOrderedFromSet(FieldSet set) {
if (set instanceof FieldList) {
return (FieldList) set;
} else {
Expand All @@ -43,15 +43,15 @@ public static final FieldList createOrderedFromSet(FieldSet set) {
}
}

public static final Ordering createOrdering(FieldList fields, boolean[] directions) {
public static Ordering createOrdering(FieldList fields, boolean[] directions) {
final Ordering o = new Ordering();
for (int i = 0; i < fields.size(); i++) {
o.appendOrdering(fields.get(i), null, directions == null || directions[i] ? Order.ASCENDING : Order.DESCENDING);
}
return o;
}

public static final Ordering createOrdering(FieldList fields) {
public static Ordering createOrdering(FieldList fields) {
final Ordering o = new Ordering();
for (int i = 0; i < fields.size(); i++) {
o.appendOrdering(fields.get(i), null, Order.ANY);
Expand All @@ -77,5 +77,7 @@ public static boolean[] getDirections(Ordering o, int numFields) {
/**
* No instantiation.
*/
private Utils() {}
private Utils() {
throw new RuntimeException();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,9 @@

import static com.google.common.base.Preconditions.checkNotNull;

/**
* Utility class to work with blob data.
*/
public class BlobUtils {

/**
Expand Down Expand Up @@ -322,4 +325,11 @@ static void closeSilently(Socket socket, Logger LOG) {
}
}
}

/**
* Private constructor to prevent instantiation.
*/
private BlobUtils() {
throw new RuntimeException();
}
}
Loading

0 comments on commit 136347d

Please sign in to comment.