You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2017/03/30 22:04:42 UTC

[31/50] [abbrv] flink git commit: [FLINK-5911] [gelly] Command-line parameters

[FLINK-5911] [gelly] Command-line parameters

Create interface for parsing command-line parameters using ParameterTool
and generic implementations for boolean, long, double, string, choice.

This closes #3433


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f2a8bc92
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f2a8bc92
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f2a8bc92

Branch: refs/heads/table-retraction
Commit: f2a8bc92df583c0a6beb519fd8a84cb40edbd060
Parents: 037b5cb
Author: Greg Hogan <co...@greghogan.com>
Authored: Tue Feb 28 12:35:07 2017 -0500
Committer: Greg Hogan <co...@greghogan.com>
Committed: Mon Mar 27 06:53:29 2017 -0400

----------------------------------------------------------------------
 .../drivers/parameter/BooleanParameter.java     |  54 +++
 .../drivers/parameter/ChoiceParameter.java      | 113 ++++++
 .../drivers/parameter/DoubleParameter.java      | 177 ++++++++++
 .../graph/drivers/parameter/LongParameter.java  | 129 +++++++
 .../graph/drivers/parameter/Parameter.java      |  55 +++
 .../drivers/parameter/ParameterizedBase.java    |  67 ++++
 .../drivers/parameter/SimpleParameter.java      |  72 ++++
 .../drivers/parameter/StringParameter.java      |  60 ++++
 .../flink/graph/drivers/parameter/Util.java     |  54 +++
 .../drivers/parameter/BooleanParameterTest.java |  53 +++
 .../drivers/parameter/ChoiceParameterTest.java  |  84 +++++
 .../drivers/parameter/DoubleParameterTest.java  | 341 +++++++++++++++++++
 .../drivers/parameter/LongParameterTest.java    | 223 ++++++++++++
 .../drivers/parameter/ParameterTestBase.java    |  39 +++
 .../drivers/parameter/StringParameterTest.java  |  78 +++++
 15 files changed, 1599 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java
new file mode 100644
index 0000000..7d05dbf
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/BooleanParameter.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link Boolean}.
+ */
+public class BooleanParameter
+extends SimpleParameter<Boolean> {
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public BooleanParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	@Override
+	public String getUsage() {
+		return "[--" + name + "]";
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = parameterTool.has(name);
+	}
+
+	@Override
+	public String toString() {
+		return Boolean.toString(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java
new file mode 100644
index 0000000..b239b93
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ChoiceParameter.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.commons.lang3.text.StrBuilder;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A {@link Parameter} storing a list of {@link String} choices and parsing
+ * the user's configured selection.
+ */
+public class ChoiceParameter
+extends SimpleParameter<String> {
+
+	private List<String> choices = new ArrayList<>();
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public ChoiceParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value and add to the list of choices.
+	 *
+	 * @param defaultValue the default value.
+	 * @return this
+	 */
+	public ChoiceParameter setDefaultValue(String defaultValue) {
+		super.setDefaultValue(defaultValue);
+		choices.add(defaultValue);
+		return this;
+	}
+
+	/**
+	 * Add additional choices. This function can be called multiple times.
+	 *
+	 * @param choices additional choices
+	 * @return this
+	 */
+	public ChoiceParameter addChoices(String... choices) {
+		Collections.addAll(this.choices, choices);
+		return this;
+	}
+
+	@Override
+	public String getUsage() {
+		String option = new StrBuilder()
+			.append("--")
+			.append(name)
+			.append(" <")
+			.append(StringUtils.join(choices, " | "))
+			.append(">")
+			.toString();
+
+		return hasDefaultValue ? "[" + option + "]" : option;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		Preconditions.checkArgument(choices.size() > 0, "No choices provided");
+
+		String selected = parameterTool.get(name);
+
+		if (selected == null) {
+			if (hasDefaultValue) {
+				value = defaultValue;
+				return;
+			} else {
+				throw new ProgramParametrizationException(
+					"Must select a choice for option '" + name + "': '[" + StringUtils.join(choices, ", ") + "]'");
+			}
+		}
+
+		for (String choice : choices) {
+			if (choice.equals(selected)) {
+				this.value = selected;
+				return;
+			}
+		}
+
+		throw new ProgramParametrizationException(
+			"Selection '" + selected + "' for option '" + name + "' is not in choices '[" + StringUtils.join(choices, ", ") + "]'");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java
new file mode 100644
index 0000000..78753a2
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/DoubleParameter.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link Double}.
+ */
+public class DoubleParameter
+extends SimpleParameter<Double> {
+
+	private boolean hasMinimumValue = false;
+	private boolean minimumValueInclusive;
+	private double minimumValue;
+
+	private boolean hasMaximumValue = false;
+	private boolean maximumValueInclusive;
+	private double maximumValue;
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public DoubleParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value.
+	 *
+	 * @param defaultValue the default value
+	 * @return this
+	 */
+	public DoubleParameter setDefaultValue(double defaultValue) {
+		super.setDefaultValue(defaultValue);
+
+		if (hasMinimumValue) {
+			if (minimumValueInclusive) {
+				Util.checkParameter(defaultValue >= minimumValue,
+					"Default value (" + defaultValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+			} else {
+				Util.checkParameter(defaultValue > minimumValue,
+					"Default value (" + defaultValue + ") must be greater than minimum (" + minimumValue + ")");
+			}
+		}
+
+		if (hasMaximumValue) {
+			if (maximumValueInclusive) {
+				Util.checkParameter(defaultValue <= maximumValue,
+					"Default value (" + defaultValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+			} else {
+				Util.checkParameter(defaultValue < maximumValue,
+					"Default value (" + defaultValue + ") must be less than maximum (" + maximumValue + ")");
+			}
+		}
+
+		return this;
+	}
+
+	/**
+	 * Set the minimum value. The minimum value is an acceptable value if and
+	 * only if inclusive is set to true.
+	 *
+	 * @param minimumValue the minimum value
+	 * @param inclusive whether the minimum value is a valid value
+	 * @return this
+	 */
+	public DoubleParameter setMinimumValue(double minimumValue, boolean inclusive) {
+		if (hasDefaultValue) {
+			if (inclusive) {
+				Util.checkParameter(minimumValue <= defaultValue,
+					"Minimum value (" + minimumValue + ") must be less than or equal to default (" + defaultValue + ")");
+			} else {
+				Util.checkParameter(minimumValue < defaultValue,
+					"Minimum value (" + minimumValue + ") must be less than default (" + defaultValue + ")");
+			}
+		} else if (hasMaximumValue) {
+			if (inclusive && maximumValueInclusive) {
+				Util.checkParameter(minimumValue <= maximumValue,
+					"Minimum value (" + minimumValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+			} else {
+				Util.checkParameter(minimumValue < maximumValue,
+					"Minimum value (" + minimumValue + ") must be less than maximum (" + maximumValue + ")");
+			}
+		}
+
+		this.hasMinimumValue = true;
+		this.minimumValue = minimumValue;
+		this.minimumValueInclusive = inclusive;
+
+		return this;
+	}
+
+	/**
+	 * Set the maximum value. The maximum value is an acceptable value if and
+	 * only if inclusive is set to true.
+	 *
+	 * @param maximumValue the maximum value
+	 * @param inclusive whether the maximum value is a valid value
+	 * @return this
+	 */
+	public DoubleParameter setMaximumValue(double maximumValue, boolean inclusive) {
+		if (hasDefaultValue) {
+			if (inclusive) {
+				Util.checkParameter(maximumValue >= defaultValue,
+					"Maximum value (" + maximumValue + ") must be greater than or equal to default (" + defaultValue + ")");
+			} else {
+				Util.checkParameter(maximumValue > defaultValue,
+					"Maximum value (" + maximumValue + ") must be greater than default (" + defaultValue + ")");
+			}
+		} else if (hasMinimumValue) {
+			if (inclusive && minimumValueInclusive) {
+				Util.checkParameter(maximumValue >= minimumValue,
+					"Maximum value (" + maximumValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+			} else {
+				Util.checkParameter(maximumValue > minimumValue,
+					"Maximum value (" + maximumValue + ") must be greater than minimum (" + minimumValue + ")");
+			}
+		}
+
+		this.hasMaximumValue = true;
+		this.maximumValue = maximumValue;
+		this.maximumValueInclusive = inclusive;
+
+		return this;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = hasDefaultValue ? parameterTool.getDouble(name, defaultValue) : parameterTool.getDouble(name);
+
+		if (hasMinimumValue) {
+			if (minimumValueInclusive) {
+				Util.checkParameter(value >= minimumValue,
+					name + " must be greater than or equal to " + minimumValue);
+			} else {
+				Util.checkParameter(value > minimumValue,
+					name + " must be greater than " + minimumValue);
+			}
+		}
+
+		if (hasMaximumValue) {
+			if (maximumValueInclusive) {
+				Util.checkParameter(value <= maximumValue,
+					name + " must be less than or equal to " + maximumValue);
+			} else {
+				Util.checkParameter(value < maximumValue,
+					name + " must be less than " + maximumValue);
+			}
+		}
+	}
+
+	@Override
+	public String toString() {
+		return Double.toString(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java
new file mode 100644
index 0000000..5917b14
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/LongParameter.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link Long}.
+ */
+public class LongParameter
+extends SimpleParameter<Long> {
+
+	private boolean hasMinimumValue = false;
+	private long minimumValue;
+
+	private boolean hasMaximumValue = false;
+	private long maximumValue;
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public LongParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value.
+	 *
+	 * @param defaultValue the default value.
+	 * @return this
+	 */
+	public LongParameter setDefaultValue(long defaultValue) {
+		super.setDefaultValue(defaultValue);
+
+		if (hasMinimumValue) {
+			Util.checkParameter(defaultValue >= minimumValue,
+				"Default value (" + defaultValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+		}
+
+		if (hasMaximumValue) {
+			Util.checkParameter(defaultValue <= maximumValue,
+				"Default value (" + defaultValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+		}
+
+		return this;
+	}
+
+	/**
+	 * Set the minimum value.
+	 *
+	 * @param minimumValue the minimum value
+	 * @return this
+	 */
+	public LongParameter setMinimumValue(long minimumValue) {
+		if (hasDefaultValue) {
+			Util.checkParameter(minimumValue <= defaultValue,
+				"Minimum value (" + minimumValue + ") must be less than or equal to default (" + defaultValue + ")");
+		} else if (hasMaximumValue) {
+			Util.checkParameter(minimumValue <= maximumValue,
+				"Minimum value (" + minimumValue + ") must be less than or equal to maximum (" + maximumValue + ")");
+		}
+
+		this.hasMinimumValue = true;
+		this.minimumValue = minimumValue;
+
+		return this;
+	}
+
+	/**
+	 * Set the maximum value.
+	 *
+	 * @param maximumValue the maximum value
+	 * @return this
+	 */
+	public LongParameter setMaximumValue(long maximumValue) {
+		if (hasDefaultValue) {
+			Util.checkParameter(maximumValue >= defaultValue,
+				"Maximum value (" + maximumValue + ") must be greater than or equal to default (" + defaultValue + ")");
+		} else if (hasMinimumValue) {
+			Util.checkParameter(maximumValue >= minimumValue,
+				"Maximum value (" + maximumValue + ") must be greater than or equal to minimum (" + minimumValue + ")");
+		}
+
+		this.hasMaximumValue = true;
+		this.maximumValue = maximumValue;
+
+		return this;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = hasDefaultValue ? parameterTool.getLong(name, defaultValue) : parameterTool.getLong(name);
+
+		if (hasMinimumValue) {
+			Util.checkParameter(value >= minimumValue,
+				name + " must be greater than or equal to " + minimumValue);
+		}
+
+		if (hasMaximumValue) {
+			Util.checkParameter(value <= maximumValue,
+				name + " must be less than or equal to " + maximumValue);
+		}
+	}
+
+	@Override
+	public String toString() {
+		return Long.toString(value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java
new file mode 100644
index 0000000..46785f8
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameter.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * Encapsulates the usage and configuration of a command-line parameter.
+ *
+ * @param <T> parameter value type
+ */
+public interface Parameter<T> {
+
+	/**
+	 * An informal usage string. Parameter names are prefixed with "--".
+	 *
+	 * Optional parameters are enclosed by "[" and "]".
+	 *
+	 * Generic values are represented by all-caps with specific values enclosed
+	 * by "&lt;" and "&gt;".
+	 *
+	 * @return command-line usage string
+	 */
+	String getUsage();
+
+	/**
+	 * Read and parse the parameter value from command-line arguments.
+	 *
+	 * @param parameterTool parameter parser
+	 */
+	void configure(ParameterTool parameterTool);
+
+	/**
+	 * Get the parameter value.
+	 *
+	 * @return parameter value
+	 */
+	T getValue();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
new file mode 100644
index 0000000..3b9b80a
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.commons.lang3.text.StrBuilder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Base class for a {@link Parameterized} which maintains a list of parameters
+ * used to print the command-line usage string and configure parameters.
+ */
+public abstract class ParameterizedBase
+implements Parameterized {
+
+	private List<Parameter<?>> parameters = new ArrayList<>();
+
+	/**
+	 * Adds a parameter to the list. Parameter order is preserved when printing
+	 * the command-line usage string.
+	 *
+	 * @param parameter to add to the list of parameters
+	 */
+	public void addParameter(Parameter<?> parameter) {
+		parameters.add(parameter);
+	}
+
+	@Override
+	public String getParameterization() {
+		StrBuilder strBuilder = new StrBuilder();
+
+		// print parameters as ordered list
+		for (Parameter<?> parameter : parameters) {
+			strBuilder
+				.append(parameter.getUsage())
+				.append(" ");
+		}
+
+		return strBuilder.toString();
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) throws ProgramParametrizationException {
+		for (Parameter<?> parameter : parameters) {
+			parameter.configure(parameterTool);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java
new file mode 100644
index 0000000..93469ac
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/SimpleParameter.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+/**
+ * A {@link Parameter} with a default value.
+ */
+public abstract class SimpleParameter<T>
+implements Parameter<T> {
+
+	protected final String name;
+
+	protected boolean hasDefaultValue = false;
+
+	protected T defaultValue;
+
+	protected T value;
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	protected SimpleParameter(ParameterizedBase owner, String name) {
+		this.name = name;
+		owner.addParameter(this);
+	}
+
+	/**
+	 * Set the default value, used if no value is set by the command-line
+	 * configuration.
+	 *
+	 * @param defaultValue the default value
+	 * @return this
+	 */
+	protected SimpleParameter setDefaultValue(T defaultValue) {
+		this.hasDefaultValue = true;
+		this.defaultValue = defaultValue;
+
+		return this;
+	}
+
+	@Override
+	public String getUsage() {
+		String option = "--" + name + " " + name.toUpperCase();
+
+		return hasDefaultValue ? "[" + option + "]" : option;
+	}
+
+	@Override
+	public T getValue() {
+		return value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java
new file mode 100644
index 0000000..34194ec
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/StringParameter.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+
+/**
+ * A {@link Parameter} storing a {@link String}.
+ */
+public class StringParameter
+extends SimpleParameter<String> {
+
+	/**
+	 * Set the parameter name and add this parameter to the list of parameters
+	 * stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param name the parameter name
+	 */
+	public StringParameter(ParameterizedBase owner, String name) {
+		super(owner, name);
+	}
+
+	/**
+	 * Set the default value.
+	 *
+	 * @param defaultValue the default value.
+	 * @return this
+	 */
+	public StringParameter setDefaultValue(String defaultValue) {
+		super.setDefaultValue(defaultValue);
+		return this;
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		value = hasDefaultValue ? parameterTool.get(name, defaultValue) : parameterTool.getRequired(name);
+	}
+
+	@Override
+	public String toString() {
+		return value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java
new file mode 100644
index 0000000..ccc290e
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Util.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+/**
+ * Utility methods for parsing command-line arguments.
+ */
+public class Util {
+
+	private Util() {}
+
+	// ------------------------------------------------------------------------
+	//  Boolean Condition Checking (Argument)
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Checks the given boolean condition, and throws an {@code ProgramParametrizationException} if
+	 * the condition is not met (evaluates to {@code false}). The exception will have the
+	 * given error message.
+	 *
+	 * @param condition The condition to check
+	 * @param errorMessage The message for the {@code ProgramParametrizationException} that is thrown if the check fails.
+	 *
+	 * @throws ProgramParametrizationException Thrown, if the condition is violated.
+	 *
+	 * @see Preconditions#checkNotNull(Object, String)
+	 */
+	public static void checkParameter(boolean condition, @Nullable Object errorMessage) {
+		if (!condition) {
+			throw new ProgramParametrizationException(String.valueOf(errorMessage));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java
new file mode 100644
index 0000000..43bba88
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/BooleanParameterTest.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class BooleanParameterTest
+extends ParameterTestBase {
+
+	private BooleanParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new BooleanParameter(owner, "test");
+	}
+
+	@Test
+	public void testTrue() {
+		Assert.assertEquals("[--test]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test"}));
+		Assert.assertEquals(true, parameter.getValue());
+	}
+
+	@Test
+	public void testFalse() {
+		Assert.assertEquals("[--test]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(false, parameter.getValue());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java
new file mode 100644
index 0000000..1ed1af3
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ChoiceParameterTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class ChoiceParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private ChoiceParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+		parameter = new ChoiceParameter(owner, "choice");
+	}
+
+	// With default
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue("default").addChoices("c0", "c1", "c2");
+		Assert.assertEquals("[--choice <default | c0 | c1 | c2>]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--choice", "c1"}));
+		Assert.assertEquals("c1", parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue("default").addChoices("c0", "c1", "c2");
+		Assert.assertEquals("[--choice <default | c0 | c1 | c2>]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals("default", parameter.getValue());
+	}
+
+	// Without default
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		parameter.addChoices("c0", "c1", "c2");
+		Assert.assertEquals("--choice <c0 | c1 | c2>", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--choice", "c2"}));
+		Assert.assertEquals("c2", parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		parameter.addChoices("c0", "c1", "c2");
+		Assert.assertEquals("--choice <c0 | c1 | c2>", parameter.getUsage());
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Must select a choice for option 'choice': '[c0, c1, c2]'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java
new file mode 100644
index 0000000..7a4d4fa
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/DoubleParameterTest.java
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class DoubleParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private DoubleParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new DoubleParameter(owner, "test");
+	}
+
+	// Test configuration
+
+	@Test
+	public void testDefaultValueBelowMinimum() {
+		parameter.setMinimumValue(1.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0.0) must be greater than minimum (1.0)");
+
+		parameter.setDefaultValue(0.0);
+	}
+
+	@Test
+	public void testDefaultValueBetweenMinAndMax() {
+		parameter.setMinimumValue(-1.0, false);
+		parameter.setMaximumValue(1.0, false);
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testDefaultValueAboveMaximum() {
+		parameter.setMaximumValue(-1.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0.0) must be less than maximum (-1.0)");
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testMinimumValueAboveMaximum() {
+		parameter.setMaximumValue(0.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1.0) must be less than maximum (0.0)");
+
+		parameter.setMinimumValue(1.0, false);
+	}
+
+	@Test
+	public void testMinimumValueAboveDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1.0) must be less than default (0.0)");
+
+		parameter.setMinimumValue(1.0, false);
+	}
+
+	@Test
+	public void testMaximumValueBelowMinimum() {
+		parameter.setMinimumValue(0.0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1.0) must be greater than minimum (0.0)");
+
+		parameter.setMaximumValue(-1.0, false);
+	}
+
+	@Test
+	public void testMaximumValueBelowDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1.0) must be greater than default (0.0)");
+
+		parameter.setMaximumValue(-1.0, false);
+	}
+
+	@Test
+	public void testEqualMinimumAndMaximumInclusive() {
+		parameter.setMinimumValue(0.0, true);
+		parameter.setMaximumValue(0.0, true);
+	}
+
+	@Test
+	public void testMinimumEqualsMaximumExclusive() {
+		parameter.setMaximumValue(0.0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (0.0) must be less than maximum (0.0)");
+
+		parameter.setMinimumValue(0.0, false);
+	}
+
+	@Test
+	public void testMaximumEqualsMinimumExclusive() {
+		parameter.setMinimumValue(0.0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (0.0) must be greater than minimum (0.0)");
+
+		parameter.setMaximumValue(0.0, false);
+	}
+
+	// With default
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue(43.21);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "12.34"}));
+		Assert.assertEquals(new Double(12.34), parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue(43.21);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(new Double(43.21), parameter.getValue());
+	}
+
+	// Without default
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "12.34"}));
+		Assert.assertEquals(new Double(12.34), parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		expectedException.expect(RuntimeException.class);
+		expectedException.expectMessage("No data for required key 'test'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+
+	// Min
+
+	@Test
+	public void testMinInRange() {
+		parameter.setMinimumValue(0, false);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+		Assert.assertEquals(new Double(1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAtRangeInclusive() {
+		parameter.setMinimumValue(0, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Double(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAtRangeExclusive() {
+		parameter.setMinimumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+	}
+
+	@Test
+	public void testMinOutOfRange() {
+		parameter.setMinimumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	@Test
+	public void testMinOutOfRangeExclusive() {
+		parameter.setMinimumValue(0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than or equal to 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	// Max
+
+	@Test
+	public void testMaxInRange() {
+		parameter.setMaximumValue(0, false);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+		Assert.assertEquals(new Double(-1), parameter.getValue());
+	}
+
+	@Test
+	public void testMaxAtRangeInclusive() {
+		parameter.setMaximumValue(0, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Double(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMaxAtRangeExclusive() {
+		parameter.setMaximumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+	}
+
+	@Test
+	public void testMaxOutOfRange() {
+		parameter.setMaximumValue(0, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	@Test
+	public void testMaxOutOfRangeExclusive() {
+		parameter.setMaximumValue(0, true);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than or equal to 0.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	// Min and max
+
+	@Test
+	public void testMinAndMaxBelowRange() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than -1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-2"}));
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMinimumExclusive() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than -1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMinimumInclusive() {
+		parameter.setMinimumValue(-1, true);
+		parameter.setMaximumValue(1, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+		Assert.assertEquals(new Double(-1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxInRange() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Double(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMaximumInclusive() {
+		parameter.setMinimumValue(-1, true);
+		parameter.setMaximumValue(1, true);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+		Assert.assertEquals(new Double(1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxAtRangeMaximumExclusive() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	@Test
+	public void testMinAndMaxAboveRange() {
+		parameter.setMinimumValue(-1, false);
+		parameter.setMaximumValue(1, false);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than 1.0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "2"}));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java
new file mode 100644
index 0000000..2c26268
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/LongParameterTest.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class LongParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private LongParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new LongParameter(owner, "test");
+	}
+
+	// Test configuration
+
+	@Test
+	public void testDefaultValueBelowMinimum() {
+		parameter.setMinimumValue(1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0) must be greater than or equal to minimum (1)");
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testDefaultValueBetweenMinAndMax() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testDefaultValueAboveMaximum() {
+		parameter.setMaximumValue(-1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Default value (0) must be less than or equal to maximum (-1)");
+
+		parameter.setDefaultValue(0);
+	}
+
+	@Test
+	public void testMinimumValueAboveMaximum() {
+		parameter.setMaximumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1) must be less than or equal to maximum (0)");
+
+		parameter.setMinimumValue(1);
+	}
+
+	@Test
+	public void testMinimumValueAboveDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Minimum value (1) must be less than or equal to default (0)");
+
+		parameter.setMinimumValue(1);
+	}
+
+	@Test
+	public void testMaximumValueBelowMinimum() {
+		parameter.setMinimumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1) must be greater than or equal to minimum (0)");
+
+		parameter.setMaximumValue(-1);
+	}
+
+	@Test
+	public void testMaximumValueBelowDefault() {
+		parameter.setDefaultValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("Maximum value (-1) must be greater than or equal to default (0)");
+
+		parameter.setMaximumValue(-1);
+	}
+
+	// With default
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue(42);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "54"}));
+		Assert.assertEquals(new Long(54), parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue(13);
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(new Long(13), parameter.getValue());
+	}
+
+	// Without default
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "42"}));
+		Assert.assertEquals(new Long(42), parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		expectedException.expect(RuntimeException.class);
+		expectedException.expectMessage("No data for required key 'test'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+
+	// Min
+
+	@Test
+	public void testMinInRange() {
+		parameter.setMinimumValue(0);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+		Assert.assertEquals(new Long(1), parameter.getValue());
+	}
+
+	@Test
+	public void testMinOutOfRange() {
+		parameter.setMinimumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than or equal to 0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+	}
+
+	// Max
+
+	@Test
+	public void testMaxInRange() {
+		parameter.setMaximumValue(0);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-1"}));
+		Assert.assertEquals(new Long(-1), parameter.getValue());
+	}
+
+	@Test
+	public void testMaxOutOfRange() {
+		parameter.setMaximumValue(0);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than or equal to 0");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "1"}));
+	}
+
+	// Min and max
+
+	@Test
+	public void testMinAndMaxBelowRange() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be greater than or equal to -1");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "-2"}));
+	}
+
+	@Test
+	public void testMinAndMaxInRange() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "0"}));
+		Assert.assertEquals(new Long(0), parameter.getValue());
+	}
+
+	@Test
+	public void testMinAndMaxAboveRange() {
+		parameter.setMinimumValue(-1);
+		parameter.setMaximumValue(1);
+
+		expectedException.expect(ProgramParametrizationException.class);
+		expectedException.expectMessage("test must be less than or equal to 1");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "2"}));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java
new file mode 100644
index 0000000..a5dc0c6
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/ParameterTestBase.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.junit.Before;
+
+public class ParameterTestBase {
+
+	protected ParameterizedBase owner;
+
+	@Before
+	public void setup() {
+		owner = new MockParameterized();
+	}
+
+	protected static class MockParameterized
+	extends ParameterizedBase {
+		@Override
+		public String getName() {
+			return MockParameterized.class.getSimpleName();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2a8bc92/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java
new file mode 100644
index 0000000..496d85c
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/StringParameterTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class StringParameterTest
+extends ParameterTestBase {
+
+	@Rule
+	public ExpectedException expectedException = ExpectedException.none();
+
+	private StringParameter parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new StringParameter(owner, "test");
+	}
+
+	@Test
+	public void testWithDefaultWithParameter() {
+		parameter.setDefaultValue("Flink");
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "Gelly"}));
+		Assert.assertEquals("Gelly", parameter.getValue());
+	}
+
+	@Test
+	public void testWithDefaultWithoutParameter() {
+		parameter.setDefaultValue("Flink");
+		Assert.assertEquals("[--test TEST]", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals("Flink", parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--test", "Gelly"}));
+		Assert.assertEquals("Gelly", parameter.getValue());
+	}
+
+	@Test
+	public void testWithoutDefaultWithoutParameter() {
+		Assert.assertEquals("--test TEST", parameter.getUsage());
+
+		expectedException.expect(RuntimeException.class);
+		expectedException.expectMessage("No data for required key 'test'");
+
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+	}
+}